ソースを参照

HDFS-14304: High lock contention on hdfsHashMutex in libhdfs

This closes #595

Signed-off-by: Todd Lipcon <todd@apache.org>
Sahil Takiar 6 年 前
コミット
18c57cf046
16 ファイル変更636 行追加920 行削除
  1. 1 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt
  2. 18 24
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/native_mini_dfs.c
  3. 0 100
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_htable.c
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
  5. 0 287
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/common/htable.c
  6. 0 161
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/common/htable.h
  7. 3 3
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/exception.c
  8. 206 217
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c
  9. 136 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.c
  10. 112 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h
  11. 121 102
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c
  12. 27 10
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.h
  13. 3 3
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/mutexes.h
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/posix/mutexes.c
  15. 5 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/posix/thread_local_storage.c
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/mutexes.c

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt

@@ -29,8 +29,8 @@ include_directories(
 
 add_library(native_mini_dfs
     native_mini_dfs.c
-    ../libhdfs/common/htable.c
     ../libhdfs/exception.c
+    ../libhdfs/jclasses.c
     ../libhdfs/jni_helper.c
     ${OS_DIR}/mutexes.c
     ${OS_DIR}/thread_local_storage.c
@@ -39,6 +39,3 @@ add_library(native_mini_dfs
 add_executable(test_native_mini_dfs test_native_mini_dfs.c)
 target_link_libraries(test_native_mini_dfs native_mini_dfs ${JAVA_JVM_LIBRARY})
 add_test(test_test_native_mini_dfs test_native_mini_dfs)
-
-add_executable(test_htable ../libhdfs/common/htable.c test_htable.c)
-target_link_libraries(test_htable ${OS_LINK_LIBRARIES})

+ 18 - 24
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/native_mini_dfs.c

@@ -17,6 +17,7 @@
  */
 
 #include "exception.h"
+#include "jclasses.h"
 #include "jni_helper.h"
 #include "native_mini_dfs.h"
 #include "platform.h"
@@ -36,9 +37,7 @@
 
 #define MINIDFS_CLUSTER_BUILDER "org/apache/hadoop/hdfs/MiniDFSCluster$Builder"
 #define MINIDFS_CLUSTER "org/apache/hadoop/hdfs/MiniDFSCluster"
-#define HADOOP_CONF     "org/apache/hadoop/conf/Configuration"
 #define HADOOP_NAMENODE "org/apache/hadoop/hdfs/server/namenode/NameNode"
-#define JAVA_INETSOCKETADDRESS "java/net/InetSocketAddress"
 
 struct NativeMiniDfsCluster {
     /**
@@ -60,8 +59,7 @@ static int hdfsDisableDomainSocketSecurity(void)
       errno = EINTERNAL;
       return -1;
     }
-    jthr = invokeMethod(env, NULL, STATIC, NULL,
-            "org/apache/hadoop/net/unix/DomainSocket",
+    jthr = invokeMethod(env, NULL, STATIC, NULL, JC_DOMAIN_SOCKET,
             "disableBindPathValidation", "()V");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -126,11 +124,6 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
             "nmdCreate: new Configuration");
         goto error;
     }
-    if (jthr) {
-        printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
-                              "nmdCreate: Configuration::setBoolean");
-        goto error;
-    }
     // Disable 'minimum block size' -- it's annoying in tests.
     (*env)->DeleteLocalRef(env, jconfStr);
     jconfStr = NULL;
@@ -140,8 +133,9 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
                               "nmdCreate: new String");
         goto error;
     }
-    jthr = invokeMethod(env, NULL, INSTANCE, cobj, HADOOP_CONF,
-                        "setLong", "(Ljava/lang/String;J)V", jconfStr, 0LL);
+    jthr = invokeMethod(env, NULL, INSTANCE, cobj,
+            JC_CONFIGURATION, "setLong", "(Ljava/lang/String;J)V", jconfStr,
+            0LL);
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                               "nmdCreate: Configuration::setLong");
@@ -163,7 +157,7 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
             goto error;
         }
     }
-    jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
+    jthr = findClassAndInvokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
             "format", "(Z)L" MINIDFS_CLUSTER_BUILDER ";", conf->doFormat);
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "nmdCreate: "
@@ -172,7 +166,7 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
     }
     (*env)->DeleteLocalRef(env, val.l);
     if (conf->webhdfsEnabled) {
-        jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
+        jthr = findClassAndInvokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
                         "nameNodeHttpPort", "(I)L" MINIDFS_CLUSTER_BUILDER ";",
                         conf->namenodeHttpPort);
         if (jthr) {
@@ -183,7 +177,7 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
         (*env)->DeleteLocalRef(env, val.l);
     }
     if (conf->numDataNodes) {
-        jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
+        jthr = findClassAndInvokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
                 "numDataNodes", "(I)L" MINIDFS_CLUSTER_BUILDER ";", conf->numDataNodes);
         if (jthr) {
             printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "nmdCreate: "
@@ -192,7 +186,7 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
         }
     }
     (*env)->DeleteLocalRef(env, val.l);
-    jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
+    jthr = findClassAndInvokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER,
             "build", "()L" MINIDFS_CLUSTER ";");
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -242,7 +236,7 @@ int nmdShutdown(struct NativeMiniDfsCluster* cl)
         fprintf(stderr, "nmdShutdown: getJNIEnv failed\n");
         return -EIO;
     }
-    jthr = invokeMethod(env, NULL, INSTANCE, cl->obj,
+    jthr = findClassAndInvokeMethod(env, NULL, INSTANCE, cl->obj,
             MINIDFS_CLUSTER, "shutdown", "()V");
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -260,7 +254,7 @@ int nmdWaitClusterUp(struct NativeMiniDfsCluster *cl)
         fprintf(stderr, "nmdWaitClusterUp: getJNIEnv failed\n");
         return -EIO;
     }
-    jthr = invokeMethod(env, NULL, INSTANCE, cl->obj,
+    jthr = findClassAndInvokeMethod(env, NULL, INSTANCE, cl->obj,
             MINIDFS_CLUSTER, "waitClusterUp", "()V");
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -282,7 +276,7 @@ int nmdGetNameNodePort(const struct NativeMiniDfsCluster *cl)
     }
     // Note: this will have to be updated when HA nativeMiniDfs clusters are
     // supported
-    jthr = invokeMethod(env, &jVal, INSTANCE, cl->obj,
+    jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, cl->obj,
             MINIDFS_CLUSTER, "getNameNodePort", "()I");
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -307,7 +301,7 @@ int nmdGetNameNodeHttpAddress(const struct NativeMiniDfsCluster *cl,
         return -EIO;
     }
     // First get the (first) NameNode of the cluster
-    jthr = invokeMethod(env, &jVal, INSTANCE, cl->obj, MINIDFS_CLUSTER,
+    jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, cl->obj, MINIDFS_CLUSTER,
                         "getNameNode", "()L" HADOOP_NAMENODE ";");
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -318,8 +312,8 @@ int nmdGetNameNodeHttpAddress(const struct NativeMiniDfsCluster *cl,
     jNameNode = jVal.l;
 
     // Then get the http address (InetSocketAddress) of the NameNode
-    jthr = invokeMethod(env, &jVal, INSTANCE, jNameNode, HADOOP_NAMENODE,
-                        "getHttpAddress", "()L" JAVA_INETSOCKETADDRESS ";");
+    jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, jNameNode, HADOOP_NAMENODE,
+                        "getHttpAddress", "()L" JAVA_NET_ISA ";");
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                                     "nmdGetNameNodeHttpAddress: "
@@ -328,8 +322,8 @@ int nmdGetNameNodeHttpAddress(const struct NativeMiniDfsCluster *cl,
     }
     jAddress = jVal.l;
 
-    jthr = invokeMethod(env, &jVal, INSTANCE, jAddress,
-                        JAVA_INETSOCKETADDRESS, "getPort", "()I");
+    jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, jAddress,
+                        JAVA_NET_ISA, "getPort", "()I");
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                                     "nmdGetNameNodeHttpAddress: "
@@ -338,7 +332,7 @@ int nmdGetNameNodeHttpAddress(const struct NativeMiniDfsCluster *cl,
     }
     *port = jVal.i;
 
-    jthr = invokeMethod(env, &jVal, INSTANCE, jAddress, JAVA_INETSOCKETADDRESS,
+    jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, jAddress, JAVA_NET_ISA,
                         "getHostName", "()Ljava/lang/String;");
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,

+ 0 - 100
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_htable.c

@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "common/htable.h"
-#include "expect.h"
-#include "hdfs_test.h"
-
-#include <errno.h>
-#include <inttypes.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-
-// Disable type cast and loss of precision warnings, because the test
-// manipulates void* values manually on purpose.
-#ifdef WIN32
-#pragma warning(disable: 4244 4306)
-#endif
-
-static uint32_t simple_hash(const void *key, uint32_t size)
-{
-    uintptr_t k = (uintptr_t)key;
-    return ((13 + k) * 6367) % size;
-}
-
-static int simple_compare(const void *a, const void *b)
-{
-    return a == b;
-}
-
-static void expect_102(void *f, void *k, void *v)
-{
-    int *found_102 = f;
-    uintptr_t key = (uintptr_t)k;
-    uintptr_t val = (uintptr_t)v;
-
-    if ((key == 2) && (val == 102)) {
-        *found_102 = 1;
-    } else {
-        abort();
-    }
-}
-
-static void *htable_pop_val(struct htable *ht, void *key)
-{
-    void *old_key, *old_val;
-
-    htable_pop(ht, key, &old_key, &old_val);
-    return old_val;
-}
-
-int main(void)
-{
-    struct htable *ht;
-    int found_102 = 0;
-
-    ht = htable_alloc(4, simple_hash, simple_compare);
-    EXPECT_INT_EQ(0, htable_used(ht));
-    EXPECT_INT_EQ(4, htable_capacity(ht));
-    EXPECT_NULL(htable_get(ht, (void*)123));
-    EXPECT_NULL(htable_pop_val(ht, (void*)123));
-    EXPECT_ZERO(htable_put(ht, (void*)123, (void*)456));
-    EXPECT_INT_EQ(456, (uintptr_t)htable_get(ht, (void*)123));
-    EXPECT_INT_EQ(456, (uintptr_t)htable_pop_val(ht, (void*)123));
-    EXPECT_NULL(htable_pop_val(ht, (void*)123));
-
-    // Enlarge the hash table
-    EXPECT_ZERO(htable_put(ht, (void*)1, (void*)101));
-    EXPECT_ZERO(htable_put(ht, (void*)2, (void*)102));
-    EXPECT_ZERO(htable_put(ht, (void*)3, (void*)103));
-    EXPECT_INT_EQ(3, htable_used(ht));
-    EXPECT_INT_EQ(8, htable_capacity(ht));
-    EXPECT_INT_EQ(102, (uintptr_t)htable_get(ht, (void*)2));
-    EXPECT_INT_EQ(101, (uintptr_t)htable_pop_val(ht, (void*)1));
-    EXPECT_INT_EQ(103, (uintptr_t)htable_pop_val(ht, (void*)3));
-    EXPECT_INT_EQ(1, htable_used(ht));
-    htable_visit(ht, expect_102, &found_102);
-    EXPECT_INT_EQ(1, found_102);
-    htable_free(ht);
-
-    fprintf(stderr, "SUCCESS.\n");
-    return EXIT_SUCCESS;
-}
-
-// vim: ts=4:sw=4:tw=79:et

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt

@@ -35,7 +35,7 @@ hadoop_add_dual_library(hdfs
     exception.c
     jni_helper.c
     hdfs.c
-    common/htable.c
+    jclasses.c
     ${OS_DIR}/mutexes.c
     ${OS_DIR}/thread_local_storage.c
 )

+ 0 - 287
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/common/htable.c

@@ -1,287 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "common/htable.h"
-
-#include <errno.h>
-#include <inttypes.h>
-#include <stdlib.h>
-#include <string.h>
-#include <unistd.h>
-
-struct htable_pair {
-    void *key;
-    void *val;
-};
-
-/**
- * A hash table which uses linear probing.
- */
-struct htable {
-    uint32_t capacity;
-    uint32_t used;
-    htable_hash_fn_t hash_fun;
-    htable_eq_fn_t eq_fun;
-    struct htable_pair *elem;
-};
-
-/**
- * An internal function for inserting a value into the hash table.
- *
- * Note: this function assumes that you have made enough space in the table.
- *
- * @param nelem         The new element to insert.
- * @param capacity      The capacity of the hash table.
- * @param hash_fun      The hash function to use.
- * @param key           The key to insert.
- * @param val           The value to insert.
- */
-static void htable_insert_internal(struct htable_pair *nelem, 
-        uint32_t capacity, htable_hash_fn_t hash_fun, void *key,
-        void *val)
-{
-    uint32_t i;
-
-    i = hash_fun(key, capacity);
-    while (1) {
-        if (!nelem[i].key) {
-            nelem[i].key = key;
-            nelem[i].val = val;
-            return;
-        }
-        i++;
-        if (i == capacity) {
-            i = 0;
-        }
-    }
-}
-
-static int htable_realloc(struct htable *htable, uint32_t new_capacity)
-{
-    struct htable_pair *nelem;
-    uint32_t i, old_capacity = htable->capacity;
-    htable_hash_fn_t hash_fun = htable->hash_fun;
-
-    nelem = calloc(new_capacity, sizeof(struct htable_pair));
-    if (!nelem) {
-        return ENOMEM;
-    }
-    for (i = 0; i < old_capacity; i++) {
-        struct htable_pair *pair = htable->elem + i;
-        if (pair->key) {
-            htable_insert_internal(nelem, new_capacity, hash_fun,
-                                   pair->key, pair->val);
-        }
-    }
-    free(htable->elem);
-    htable->elem = nelem;
-    htable->capacity = new_capacity;
-    return 0;
-}
-
-static uint32_t round_up_to_power_of_2(uint32_t i)
-{
-    if (i == 0) {
-        return 1;
-    }
-    i--;
-    i |= i >> 1;
-    i |= i >> 2;
-    i |= i >> 4;
-    i |= i >> 8;
-    i |= i >> 16;
-    i++;
-    return i;
-}
-
-struct htable *htable_alloc(uint32_t size,
-                htable_hash_fn_t hash_fun, htable_eq_fn_t eq_fun)
-{
-    struct htable *htable;
-
-    htable = calloc(1, sizeof(*htable));
-    if (!htable) {
-        return NULL;
-    }
-    size = round_up_to_power_of_2(size);
-    if (size < HTABLE_MIN_SIZE) {
-        size = HTABLE_MIN_SIZE;
-    }
-    htable->hash_fun = hash_fun;
-    htable->eq_fun = eq_fun;
-    htable->used = 0;
-    if (htable_realloc(htable, size)) {
-        free(htable);
-        return NULL;
-    }
-    return htable;
-}
-
-void htable_visit(struct htable *htable, visitor_fn_t fun, void *ctx)
-{
-    uint32_t i;
-
-    for (i = 0; i != htable->capacity; ++i) {
-        struct htable_pair *elem = htable->elem + i;
-        if (elem->key) {
-            fun(ctx, elem->key, elem->val);
-        }
-    }
-}
-
-void htable_free(struct htable *htable)
-{
-    if (htable) {
-        free(htable->elem);
-        free(htable);
-    }
-}
-
-int htable_put(struct htable *htable, void *key, void *val)
-{
-    int ret;
-    uint32_t nused;
-
-    // NULL is not a valid key value.
-    // This helps us implement htable_get_internal efficiently, since we know
-    // that we can stop when we encounter the first NULL key.
-    if (!key) {
-        return EINVAL;
-    }
-    // NULL is not a valid value.  Otherwise the results of htable_get would
-    // be confusing (does a NULL return mean entry not found, or that the
-    // entry was found and was NULL?) 
-    if (!val) {
-        return EINVAL;
-    }
-    // Re-hash if we have used more than half of the hash table
-    nused = htable->used + 1;
-    if (nused >= (htable->capacity / 2)) {
-        ret = htable_realloc(htable, htable->capacity * 2);
-        if (ret)
-            return ret;
-    }
-    htable_insert_internal(htable->elem, htable->capacity,
-                                htable->hash_fun, key, val);
-    htable->used++;
-    return 0;
-}
-
-static int htable_get_internal(const struct htable *htable,
-                               const void *key, uint32_t *out)
-{
-    uint32_t start_idx, idx;
-
-    start_idx = htable->hash_fun(key, htable->capacity);
-    idx = start_idx;
-    while (1) {
-        struct htable_pair *pair = htable->elem + idx;
-        if (!pair->key) {
-            // We always maintain the invariant that the entries corresponding
-            // to a given key are stored in a contiguous block, not separated
-            // by any NULLs.  So if we encounter a NULL, our search is over.
-            return ENOENT;
-        } else if (htable->eq_fun(pair->key, key)) {
-            *out = idx;
-            return 0;
-        }
-        idx++;
-        if (idx == htable->capacity) {
-            idx = 0;
-        }
-        if (idx == start_idx) {
-            return ENOENT;
-        }
-    }
-}
-
-void *htable_get(const struct htable *htable, const void *key)
-{
-    uint32_t idx;
-
-    if (htable_get_internal(htable, key, &idx)) {
-        return NULL;
-    }
-    return htable->elem[idx].val;
-}
-
-void htable_pop(struct htable *htable, const void *key,
-                void **found_key, void **found_val)
-{
-    uint32_t hole, i;
-    const void *nkey;
-
-    if (htable_get_internal(htable, key, &hole)) {
-        *found_key = NULL;
-        *found_val = NULL;
-        return;
-    }
-    i = hole;
-    htable->used--;
-    // We need to maintain the compactness invariant used in
-    // htable_get_internal.  This invariant specifies that the entries for any
-    // given key are never separated by NULLs (although they may be separated
-    // by entries for other keys.)
-    while (1) {
-        i++;
-        if (i == htable->capacity) {
-            i = 0;
-        }
-        nkey = htable->elem[i].key;
-        if (!nkey) {
-            *found_key = htable->elem[hole].key;
-            *found_val = htable->elem[hole].val;
-            htable->elem[hole].key = NULL;
-            htable->elem[hole].val = NULL;
-            return;
-        } else if (htable->eq_fun(key, nkey)) {
-            htable->elem[hole].key = htable->elem[i].key;
-            htable->elem[hole].val = htable->elem[i].val;
-            hole = i;
-        }
-    }
-}
-
-uint32_t htable_used(const struct htable *htable)
-{
-    return htable->used;
-}
-
-uint32_t htable_capacity(const struct htable *htable)
-{
-    return htable->capacity;
-}
-
-uint32_t ht_hash_string(const void *str, uint32_t max)
-{
-    const char *s = str;
-    uint32_t hash = 0;
-
-    while (*s) {
-        hash = (hash * 31) + *s;
-        s++;
-    }
-    return hash % max;
-}
-
-int ht_compare_string(const void *a, const void *b)
-{
-    return strcmp(a, b) == 0;
-}
-
-// vim: ts=4:sw=4:tw=79:et

+ 0 - 161
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/common/htable.h

@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef HADOOP_CORE_COMMON_HASH_TABLE
-#define HADOOP_CORE_COMMON_HASH_TABLE
-
-#include <inttypes.h>
-#include <stdio.h>
-#include <stdint.h>
-
-#define HTABLE_MIN_SIZE 4
-
-struct htable;
-
-/**
- * An HTable hash function.
- *
- * @param key       The key.
- * @param capacity  The total capacity.
- *
- * @return          The hash slot.  Must be less than the capacity.
- */
-typedef uint32_t (*htable_hash_fn_t)(const void *key, uint32_t capacity);
-
-/**
- * An HTable equality function.  Compares two keys.
- *
- * @param a         First key.
- * @param b         Second key.
- *
- * @return          nonzero if the keys are equal.
- */
-typedef int (*htable_eq_fn_t)(const void *a, const void *b);
-
-/**
- * Allocate a new hash table.
- *
- * @param capacity  The minimum suggested starting capacity.
- * @param hash_fun  The hash function to use in this hash table.
- * @param eq_fun    The equals function to use in this hash table.
- *
- * @return          The new hash table on success; NULL on OOM.
- */
-struct htable *htable_alloc(uint32_t capacity, htable_hash_fn_t hash_fun,
-                            htable_eq_fn_t eq_fun);
-
-typedef void (*visitor_fn_t)(void *ctx, void *key, void *val);
-
-/**
- * Visit all of the entries in the hash table.
- *
- * @param htable    The hash table.
- * @param fun       The callback function to invoke on each key and value.
- * @param ctx       Context pointer to pass to the callback.
- */
-void htable_visit(struct htable *htable, visitor_fn_t fun, void *ctx);
-
-/**
- * Free the hash table.
- *
- * It is up the calling code to ensure that the keys and values inside the
- * table are de-allocated, if that is necessary.
- *
- * @param htable    The hash table.
- */
-void htable_free(struct htable *htable);
-
-/**
- * Add an entry to the hash table.
- *
- * @param htable    The hash table.
- * @param key       The key to add.  This cannot be NULL.
- * @param fun       The value to add.  This cannot be NULL.
- *
- * @return          0 on success;
- *                  EEXIST if the value already exists in the table;
- *                  ENOMEM if there is not enough memory to add the element.
- *                  EFBIG if the hash table has too many entries to fit in 32
- *                      bits.
- */
-int htable_put(struct htable *htable, void *key, void *val);
-
-/**
- * Get an entry from the hash table.
- *
- * @param htable    The hash table.
- * @param key       The key to find.
- *
- * @return          NULL if there is no such entry; the entry otherwise.
- */
-void *htable_get(const struct htable *htable, const void *key);
-
-/**
- * Get an entry from the hash table and remove it.
- *
- * @param htable    The hash table.
- * @param key       The key for the entry find and remove.
- * @param found_key (out param) NULL if the entry was not found; the found key
- *                      otherwise.
- * @param found_val (out param) NULL if the entry was not found; the found
- *                      value otherwise.
- */
-void htable_pop(struct htable *htable, const void *key,
-                void **found_key, void **found_val);
-
-/**
- * Get the number of entries used in the hash table.
- *
- * @param htable    The hash table.
- *
- * @return          The number of entries used in the hash table.
- */
-uint32_t htable_used(const struct htable *htable);
-
-/**
- * Get the capacity of the hash table.
- *
- * @param htable    The hash table.
- *
- * @return          The capacity of the hash table.
- */
-uint32_t htable_capacity(const struct htable *htable);
-
-/**
- * Hash a string.
- *
- * @param str       The string.
- * @param max       Maximum hash value
- *
- * @return          A number less than max.
- */
-uint32_t ht_hash_string(const void *str, uint32_t max);
-
-/**
- * Compare two strings.
- *
- * @param a         The first string.
- * @param b         The second string.
- *
- * @return          1 if the strings are identical; 0 otherwise.
- */
-int ht_compare_string(const void *a, const void *b);
-
-#endif
-
-// vim: ts=4:sw=4:tw=79:et

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/exception.c

@@ -18,6 +18,7 @@
 
 #include "exception.h"
 #include "hdfs/hdfs.h"
+#include "jclasses.h"
 #include "jni_helper.h"
 #include "platform.h"
 
@@ -129,9 +130,8 @@ static char* getExceptionUtilString(JNIEnv *env, jthrowable exc, char *methodNam
     jvalue jVal;
     jstring jStr = NULL;
     char *excString = NULL;
-    jthr = invokeMethod(env, &jVal, STATIC, NULL,
-        "org/apache/commons/lang3/exception/ExceptionUtils",
-        methodName, "(Ljava/lang/Throwable;)Ljava/lang/String;", exc);
+    jthr = invokeMethod(env, &jVal, STATIC, NULL, JC_EXCEPTION_UTILS,
+            methodName, "(Ljava/lang/Throwable;)Ljava/lang/String;", exc);
     if (jthr) {
         destroyLocalReference(env, jthr);
         return NULL;

ファイルの差分が大きいため隠しています
+ 206 - 217
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c


+ 136 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.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 "exception.h"
+#include "jclasses.h"
+#include "jni_helper.h"
+#include "os/mutexes.h"
+
+#include <assert.h>
+
+/**
+ * Whether initCachedClasses has been called or not. Protected by the mutex
+ * jclassInitMutex.
+ */
+static int jclassesInitialized = 0;
+
+typedef struct {
+    jclass javaClass;
+    const char *className;
+} javaClassAndName;
+
+/**
+ * A collection of commonly used jclass objects that are used throughout
+ * libhdfs. The jclasses are loaded immediately after the JVM is created (see
+ * initCachedClasses). The array is indexed using CachedJavaClass.
+ */
+javaClassAndName cachedJavaClasses[NUM_CACHED_CLASSES];
+
+/**
+ * Helper method that creates and sets a jclass object given a class name.
+ * Returns a jthrowable on error, NULL otherwise.
+ */
+static jthrowable initCachedClass(JNIEnv *env, const char *className,
+        jclass *cachedJclass) {
+    assert(className != NULL && "Found a CachedJavaClass without a class "
+                                "name");
+    jthrowable jthr = NULL;
+    jclass tempLocalClassRef;
+    tempLocalClassRef = (*env)->FindClass(env, className);
+    if (!tempLocalClassRef) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    *cachedJclass = (jclass) (*env)->NewGlobalRef(env, tempLocalClassRef);
+    if (!*cachedJclass) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+done:
+    destroyLocalReference(env, tempLocalClassRef);
+    return jthr;
+}
+
+jthrowable initCachedClasses(JNIEnv* env) {
+    mutexLock(&jclassInitMutex);
+    if (!jclassesInitialized) {
+        // Set all the class names
+        cachedJavaClasses[JC_CONFIGURATION].className =
+                "org/apache/hadoop/conf/Configuration";
+        cachedJavaClasses[JC_PATH].className =
+                "org/apache/hadoop/fs/Path";
+        cachedJavaClasses[JC_FILE_SYSTEM].className =
+                "org/apache/hadoop/fs/FileSystem";
+        cachedJavaClasses[JC_FS_STATUS].className =
+                "org/apache/hadoop/fs/FsStatus";
+        cachedJavaClasses[JC_FILE_UTIL].className =
+                "org/apache/hadoop/fs/FileUtil";
+        cachedJavaClasses[JC_BLOCK_LOCATION].className =
+                "org/apache/hadoop/fs/BlockLocation";
+        cachedJavaClasses[JC_DFS_HEDGED_READ_METRICS].className =
+                "org/apache/hadoop/hdfs/DFSHedgedReadMetrics";
+        cachedJavaClasses[JC_DISTRIBUTED_FILE_SYSTEM].className =
+                "org/apache/hadoop/hdfs/DistributedFileSystem";
+        cachedJavaClasses[JC_FS_DATA_INPUT_STREAM].className =
+                "org/apache/hadoop/fs/FSDataInputStream";
+        cachedJavaClasses[JC_FS_DATA_OUTPUT_STREAM].className =
+                "org/apache/hadoop/fs/FSDataOutputStream";
+        cachedJavaClasses[JC_FILE_STATUS].className =
+                "org/apache/hadoop/fs/FileStatus";
+        cachedJavaClasses[JC_FS_PERMISSION].className =
+                "org/apache/hadoop/fs/permission/FsPermission";
+        cachedJavaClasses[JC_READ_STATISTICS].className =
+                "org/apache/hadoop/hdfs/ReadStatistics";
+        cachedJavaClasses[JC_HDFS_DATA_INPUT_STREAM].className =
+                "org/apache/hadoop/hdfs/client/HdfsDataInputStream";
+        cachedJavaClasses[JC_DOMAIN_SOCKET].className =
+                "org/apache/hadoop/net/unix/DomainSocket";
+        cachedJavaClasses[JC_URI].className =
+                "java/net/URI";
+        cachedJavaClasses[JC_BYTE_BUFFER].className =
+                "java/nio/ByteBuffer";
+        cachedJavaClasses[JC_ENUM_SET].className =
+                "java/util/EnumSet";
+        cachedJavaClasses[JC_EXCEPTION_UTILS].className =
+                "org/apache/commons/lang3/exception/ExceptionUtils";
+
+        // Create and set the jclass objects based on the class names set above
+        jthrowable jthr;
+        int numCachedClasses =
+                sizeof(cachedJavaClasses) / sizeof(javaClassAndName);
+        for (int i = 0; i < numCachedClasses; i++) {
+            jthr = initCachedClass(env, cachedJavaClasses[i].className,
+                                   &cachedJavaClasses[i].javaClass);
+            if (jthr) {
+                mutexUnlock(&jclassInitMutex);
+                return jthr;
+            }
+        }
+        jclassesInitialized = 1;
+    }
+    mutexUnlock(&jclassInitMutex);
+    return NULL;
+}
+
+jclass getJclass(CachedJavaClass cachedJavaClass) {
+    return cachedJavaClasses[cachedJavaClass].javaClass;
+}
+
+const char *getClassName(CachedJavaClass cachedJavaClass) {
+    return cachedJavaClasses[cachedJavaClass].className;
+}

+ 112 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h

@@ -0,0 +1,112 @@
+/**
+ * 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 LIBHDFS_JCLASSES_H
+#define LIBHDFS_JCLASSES_H
+
+#include <jni.h>
+
+/**
+ * Encapsulates logic to cache jclass objects so they can re-used across
+ * calls to FindClass. Creating jclass objects every time libhdfs has to
+ * invoke a method can hurt performance. By cacheing jclass objects we avoid
+ * this overhead.
+ *
+ * We use the term "cached" here loosely; jclasses are not truly cached,
+ * instead they are created once during JVM load and are kept alive until the
+ * process shutdowns. There is no eviction of jclass objects.
+ *
+ * @see https://www.ibm.com/developerworks/library/j-jni/index.html#notc
+ */
+
+/**
+ * Each enum value represents one jclass that is cached. Enum values should
+ * be passed to getJclass or getName to get the jclass object or class name
+ * represented by the enum value.
+ */
+typedef enum {
+    JC_CONFIGURATION,
+    JC_PATH,
+    JC_FILE_SYSTEM,
+    JC_FS_STATUS,
+    JC_FILE_UTIL,
+    JC_BLOCK_LOCATION,
+    JC_DFS_HEDGED_READ_METRICS,
+    JC_DISTRIBUTED_FILE_SYSTEM,
+    JC_FS_DATA_INPUT_STREAM,
+    JC_FS_DATA_OUTPUT_STREAM,
+    JC_FILE_STATUS,
+    JC_FS_PERMISSION,
+    JC_READ_STATISTICS,
+    JC_HDFS_DATA_INPUT_STREAM,
+    JC_DOMAIN_SOCKET,
+    JC_URI,
+    JC_BYTE_BUFFER,
+    JC_ENUM_SET,
+    JC_EXCEPTION_UTILS,
+    // A special marker enum that counts the number of cached jclasses
+    NUM_CACHED_CLASSES
+} CachedJavaClass;
+
+/**
+ * Internally initializes all jclass objects listed in the CachedJavaClass
+ * enum. This method is idempotent and thread-safe.
+ */
+jthrowable initCachedClasses(JNIEnv* env);
+
+/**
+ * Return the jclass object represented by the given CachedJavaClass
+ */
+jclass getJclass(CachedJavaClass cachedJavaClass);
+
+/**
+ * Return the class name represented by the given CachedJavaClass
+ */
+const char *getClassName(CachedJavaClass cachedJavaClass);
+
+/* Some frequently used HDFS class names */
+#define HADOOP_CONF     "org/apache/hadoop/conf/Configuration"
+#define HADOOP_PATH     "org/apache/hadoop/fs/Path"
+#define HADOOP_LOCALFS  "org/apache/hadoop/fs/LocalFileSystem"
+#define HADOOP_FS       "org/apache/hadoop/fs/FileSystem"
+#define HADOOP_FSSTATUS "org/apache/hadoop/fs/FsStatus"
+#define HADOOP_FILEUTIL "org/apache/hadoop/fs/FileUtil"
+#define HADOOP_BLK_LOC  "org/apache/hadoop/fs/BlockLocation"
+#define HADOOP_DFS_HRM  "org/apache/hadoop/hdfs/DFSHedgedReadMetrics"
+#define HADOOP_DFS      "org/apache/hadoop/hdfs/DistributedFileSystem"
+#define HADOOP_FSDISTRM "org/apache/hadoop/fs/FSDataInputStream"
+#define HADOOP_FSDOSTRM "org/apache/hadoop/fs/FSDataOutputStream"
+#define HADOOP_FILESTAT "org/apache/hadoop/fs/FileStatus"
+#define HADOOP_FSPERM   "org/apache/hadoop/fs/permission/FsPermission"
+#define HADOOP_RSTAT    "org/apache/hadoop/hdfs/ReadStatistics"
+#define HADOOP_HDISTRM  "org/apache/hadoop/hdfs/client/HdfsDataInputStream"
+#define HADOOP_RO       "org/apache/hadoop/fs/ReadOption"
+#define HADOOP_DS       "org/apache/hadoop/net/unix/DomainSocket"
+
+/* Some frequently used Java class names */
+#define JAVA_NET_ISA    "java/net/InetSocketAddress"
+#define JAVA_NET_URI    "java/net/URI"
+#define JAVA_BYTEBUFFER "java/nio/ByteBuffer"
+#define JAVA_STRING     "java/lang/String"
+#define JAVA_ENUMSET    "java/util/EnumSet"
+
+/* Some frequently used third-party class names */
+
+#define EXCEPTION_UTILS "org/apache/commons/lang3/exception/ExceptionUtils"
+
+#endif /*LIBHDFS_JCLASSES_H*/

+ 121 - 102
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c

@@ -18,17 +18,15 @@
 
 #include "config.h"
 #include "exception.h"
+#include "jclasses.h"
 #include "jni_helper.h"
 #include "platform.h"
-#include "common/htable.h"
 #include "os/mutexes.h"
 #include "os/thread_local_storage.h"
 
 #include <stdio.h> 
 #include <string.h> 
 
-static struct htable *gClassRefHTable = NULL;
-
 /** The Native return types that methods could return */
 #define JVOID         'V'
 #define JOBJECT       'L'
@@ -42,13 +40,6 @@ static struct htable *gClassRefHTable = NULL;
 #define JFLOAT        'F'
 #define JDOUBLE       'D'
 
-
-/**
- * MAX_HASH_TABLE_ELEM: The maximum no. of entries in the hashtable.
- * It's set to 4096 to account for (classNames + No. of threads)
- */
-#define MAX_HASH_TABLE_ELEM 4096
-
 /**
  * Length of buffer for retrieving created JVMs.  (We only ever create one.)
  */
@@ -106,32 +97,27 @@ jthrowable newCStr(JNIEnv *env, jstring jstr, char **out)
     return NULL;
 }
 
-jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
-                 jobject instObj, const char *className,
-                 const char *methName, const char *methSignature, ...)
+/**
+ * Does the work to actually execute a Java method. Takes in an existing jclass
+ * object and a va_list of arguments for the Java method to be invoked.
+ */
+static jthrowable invokeMethodOnJclass(JNIEnv *env, jvalue *retval,
+        MethType methType, jobject instObj, jclass cls, const char *className,
+        const char *methName, const char *methSignature, va_list args)
 {
-    va_list args;
-    jclass cls;
     jmethodID mid;
     jthrowable jthr;
-    const char *str; 
+    const char *str;
     char returnType;
-    
-    jthr = validateMethodType(env, methType);
-    if (jthr)
-        return jthr;
-    jthr = globalClassReference(className, env, &cls);
-    if (jthr)
-        return jthr;
-    jthr = methodIdFromClass(className, methName, methSignature, 
-                            methType, env, &mid);
+
+    jthr = methodIdFromClass(cls, className, methName, methSignature, methType,
+                             env, &mid);
     if (jthr)
         return jthr;
     str = methSignature;
     while (*str != ')') str++;
     str++;
     returnType = *str;
-    va_start(args, methSignature);
     if (returnType == JOBJECT || returnType == JARRAYOBJECT) {
         jobject jobj = NULL;
         if (methType == STATIC) {
@@ -190,7 +176,6 @@ jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
         }
         retval->i = ji;
     }
-    va_end(args);
 
     jthr = (*env)->ExceptionOccurred(env);
     if (jthr) {
@@ -200,43 +185,115 @@ jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
     return NULL;
 }
 
-jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, 
-                                  const char *ctorSignature, ...)
+jthrowable findClassAndInvokeMethod(JNIEnv *env, jvalue *retval,
+        MethType methType, jobject instObj, const char *className,
+        const char *methName, const char *methSignature, ...)
 {
+    jclass cls = NULL;
+    jthrowable jthr = NULL;
+
     va_list args;
-    jclass cls;
-    jmethodID mid; 
+    va_start(args, methSignature);
+
+    jthr = validateMethodType(env, methType);
+    if (jthr) {
+        goto done;
+    }
+
+    cls = (*env)->FindClass(env, className);
+    if (!cls) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+
+    jthr = invokeMethodOnJclass(env, retval, methType, instObj, cls,
+            className, methName, methSignature, args);
+
+done:
+    va_end(args);
+    destroyLocalReference(env, cls);
+    return jthr;
+}
+
+jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
+        jobject instObj, CachedJavaClass class,
+        const char *methName, const char *methSignature, ...)
+{
+    jthrowable jthr;
+
+    va_list args;
+    va_start(args, methSignature);
+
+    jthr = invokeMethodOnJclass(env, retval, methType, instObj,
+            getJclass(class), getClassName(class), methName, methSignature,
+            args);
+
+    va_end(args);
+    return jthr;
+}
+
+static jthrowable constructNewObjectOfJclass(JNIEnv *env,
+        jobject *out, jclass cls, const char *className,
+                const char *ctorSignature, va_list args) {
+    jmethodID mid;
     jobject jobj;
     jthrowable jthr;
 
-    jthr = globalClassReference(className, env, &cls);
+    jthr = methodIdFromClass(cls, className, "<init>", ctorSignature, INSTANCE,
+            env, &mid);
     if (jthr)
         return jthr;
-    jthr = methodIdFromClass(className, "<init>", ctorSignature, 
-                            INSTANCE, env, &mid);
-    if (jthr)
-        return jthr;
-    va_start(args, ctorSignature);
     jobj = (*env)->NewObjectV(env, cls, mid, args);
-    va_end(args);
     if (!jobj)
         return getPendingExceptionAndClear(env);
     *out = jobj;
     return NULL;
 }
 
-
-jthrowable methodIdFromClass(const char *className, const char *methName, 
-                            const char *methSignature, MethType methType, 
-                            JNIEnv *env, jmethodID *out)
+jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out,
+        const char *className, const char *ctorSignature, ...)
 {
+    va_list args;
     jclass cls;
+    jthrowable jthr = NULL;
+
+    cls = (*env)->FindClass(env, className);
+    if (!cls) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+
+    va_start(args, ctorSignature);
+    jthr = constructNewObjectOfJclass(env, out, cls, className,
+            ctorSignature, args);
+    va_end(args);
+done:
+    destroyLocalReference(env, cls);
+    return jthr;
+}
+
+jthrowable constructNewObjectOfCachedClass(JNIEnv *env, jobject *out,
+        CachedJavaClass cachedJavaClass, const char *ctorSignature, ...)
+{
+    jthrowable jthr = NULL;
+    va_list args;
+    va_start(args, ctorSignature);
+
+    jthr = constructNewObjectOfJclass(env, out,
+            getJclass(cachedJavaClass), getClassName(cachedJavaClass),
+            ctorSignature, args);
+
+    va_end(args);
+    return jthr;
+}
+
+jthrowable methodIdFromClass(jclass cls, const char *className,
+        const char *methName, const char *methSignature, MethType methType,
+        JNIEnv *env, jmethodID *out)
+{
     jthrowable jthr;
     jmethodID mid = 0;
 
-    jthr = globalClassReference(className, env, &cls);
-    if (jthr)
-        return jthr;
     jthr = validateMethodType(env, methType);
     if (jthr)
         return jthr;
@@ -255,54 +312,6 @@ jthrowable methodIdFromClass(const char *className, const char *methName,
     return NULL;
 }
 
-jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out)
-{
-    jthrowable jthr = NULL;
-    jclass local_clazz = NULL;
-    jclass clazz = NULL;
-    int ret;
-
-    mutexLock(&hdfsHashMutex);
-    if (!gClassRefHTable) {
-        gClassRefHTable = htable_alloc(MAX_HASH_TABLE_ELEM, ht_hash_string,
-            ht_compare_string);
-        if (!gClassRefHTable) {
-            jthr = newRuntimeError(env, "htable_alloc failed\n");
-            goto done;
-        }
-    }
-    clazz = htable_get(gClassRefHTable, className);
-    if (clazz) {
-        *out = clazz;
-        goto done;
-    }
-    local_clazz = (*env)->FindClass(env,className);
-    if (!local_clazz) {
-        jthr = getPendingExceptionAndClear(env);
-        goto done;
-    }
-    clazz = (*env)->NewGlobalRef(env, local_clazz);
-    if (!clazz) {
-        jthr = getPendingExceptionAndClear(env);
-        goto done;
-    }
-    ret = htable_put(gClassRefHTable, (void*)className, clazz);
-    if (ret) {
-        jthr = newRuntimeError(env, "htable_put failed with error "
-                               "code %d\n", ret);
-        goto done;
-    }
-    *out = clazz;
-    jthr = NULL;
-done:
-    mutexUnlock(&hdfsHashMutex);
-    (*env)->DeleteLocalRef(env, local_clazz);
-    if (jthr && clazz) {
-        (*env)->DeleteGlobalRef(env, clazz);
-    }
-    return jthr;
-}
-
 jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name)
 {
     jthrowable jthr;
@@ -361,7 +370,6 @@ done:
     return jthr;
 }
 
-
 /**
  * Get the global JNI environemnt.
  *
@@ -461,14 +469,17 @@ static JNIEnv* getGlobalJNIEnv(void)
                     "with error: %d\n", rv);
             return NULL;
         }
-        jthr = invokeMethod(env, NULL, STATIC, NULL,
-                         "org/apache/hadoop/fs/FileSystem",
-                         "loadFileSystems", "()V");
+
+        // We use findClassAndInvokeMethod here because the jclasses in
+        // jclasses.h have not loaded yet
+        jthr = findClassAndInvokeMethod(env, NULL, STATIC, NULL, HADOOP_FS,
+                "loadFileSystems", "()V");
         if (jthr) {
-            printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "loadFileSystems");
+            printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "FileSystem: loadFileSystems failed");
+            return NULL;
         }
-    }
-    else {
+    } else {
         //Attach this thread to the VM
         vm = vmBuf[0];
         rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
@@ -539,6 +550,15 @@ JNIEnv* getJNIEnv(void)
 
     state->env = getGlobalJNIEnv();
     mutexUnlock(&jvmMutex);
+
+    jthrowable jthr = NULL;
+    jthr = initCachedClasses(state->env);
+    if (jthr) {
+      printExceptionAndFree(state->env, jthr, PRINT_EXC_ALL,
+                            "initCachedClasses failed");
+      goto fail;
+    }
+
     if (!state->env) {
       goto fail;
     }
@@ -628,8 +648,7 @@ jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
     if (jthr)
         goto done;
     jthr = invokeMethod(env, NULL, INSTANCE, jConfiguration,
-            "org/apache/hadoop/conf/Configuration", "set", 
-            "(Ljava/lang/String;Ljava/lang/String;)V",
+            JC_CONFIGURATION, "set", "(Ljava/lang/String;Ljava/lang/String;)V",
             jkey, jvalue);
     if (jthr)
         goto done;

+ 27 - 10
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.h

@@ -19,6 +19,8 @@
 #ifndef LIBHDFS_JNI_HELPER_H
 #define LIBHDFS_JNI_HELPER_H
 
+#include "jclasses.h"
+
 #include <jni.h>
 #include <stdio.h>
 
@@ -28,7 +30,6 @@
 
 #define PATH_SEPARATOR ':'
 
-
 /** Denote the method we want to invoke as STATIC or INSTANCE */
 typedef enum {
     STATIC,
@@ -66,12 +67,12 @@ jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out);
 void destroyLocalReference(JNIEnv *env, jobject jObject);
 
 /** invokeMethod: Invoke a Static or Instance method.
- * className: Name of the class where the method can be found
  * methName: Name of the method
  * methSignature: the signature of the method "(arg-types)ret-type"
  * methType: The type of the method (STATIC or INSTANCE)
  * instObj: Required if the methType is INSTANCE. The object to invoke
    the method on.
+ * class: The CachedJavaClass to call the method on.
  * env: The JNIEnv pointer
  * retval: The pointer to a union type which will contain the result of the
    method invocation, e.g. if the method returns an Object, retval will be
@@ -83,17 +84,33 @@ void destroyLocalReference(JNIEnv *env, jobject jObject);
    a valid exception reference, and the result stored at retval is undefined.
  */
 jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
-                 jobject instObj, const char *className, const char *methName, 
-                 const char *methSignature, ...);
+        jobject instObj, CachedJavaClass class,
+        const char *methName, const char *methSignature, ...);
 
-jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, 
-                                  const char *ctorSignature, ...);
+/**
+ * findClassAndInvokeMethod: Same as invokeMethod, but it calls FindClass on
+ * the given className first and then calls invokeMethod. This method exists
+ * mainly for test infrastructure, any production code should use
+ * invokeMethod. Calling FindClass repeatedly can introduce performance
+ * overhead, so users should prefer invokeMethod and supply a CachedJavaClass.
+ */
+jthrowable findClassAndInvokeMethod(JNIEnv *env, jvalue *retval,
+        MethType methType, jobject instObj, const char *className,
+        const char *methName, const char *methSignature, ...);
 
-jthrowable methodIdFromClass(const char *className, const char *methName, 
-                            const char *methSignature, MethType methType, 
-                            JNIEnv *env, jmethodID *out);
+jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out,
+        const char *className, const char *ctorSignature, ...);
+
+/**
+ * Same as constructNewObjectOfClass but it takes in a CachedJavaClass
+ * rather than a className. This avoids an extra call to FindClass.
+ */
+jthrowable constructNewObjectOfCachedClass(JNIEnv *env, jobject *out,
+        CachedJavaClass cachedJavaClass, const char *ctorSignature, ...);
 
-jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out);
+jthrowable methodIdFromClass(jclass cls, const char *className,
+        const char *methName, const char *methSignature, MethType methType,
+        JNIEnv *env, jmethodID *out);
 
 /** classNameOfObject: Get an object's class name.
  * @param jobj: The object.

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/mutexes.h

@@ -30,12 +30,12 @@
 
 #include "platform.h"
 
-/** Mutex protecting the class reference hash table. */
-extern mutex hdfsHashMutex;
-
 /** Mutex protecting singleton JVM instance. */
 extern mutex jvmMutex;
 
+/** Mutex protecting initialization of jclasses in jclasses.h. */
+extern mutex jclassInitMutex;
+
 /**
  * Locks a mutex.
  *

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/posix/mutexes.c

@@ -21,8 +21,8 @@
 #include <pthread.h>
 #include <stdio.h>
 
-mutex hdfsHashMutex = PTHREAD_MUTEX_INITIALIZER;
 mutex jvmMutex;
+mutex jclassInitMutex = PTHREAD_MUTEX_INITIALIZER;
 pthread_mutexattr_t jvmMutexAttr;
 
 __attribute__((constructor)) static void init() {

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/posix/thread_local_storage.c

@@ -97,7 +97,7 @@ static void get_current_thread_id(JNIEnv* env, char* id, int max) {
   jthrowable jthr = NULL;
   const char *thr_name_str;
 
-  jthr = invokeMethod(env, &jVal, STATIC, NULL, "java/lang/Thread",
+  jthr = findClassAndInvokeMethod(env, &jVal, STATIC, NULL, "java/lang/Thread",
           "currentThread", "()Ljava/lang/Thread;");
   if (jthr) {
     snprintf(id, max, "%s", UNKNOWN);
@@ -107,8 +107,8 @@ static void get_current_thread_id(JNIEnv* env, char* id, int max) {
   }
   thr = jVal.l;
 
-  jthr = invokeMethod(env, &jVal, INSTANCE, thr, "java/lang/Thread",
-                      "getId", "()J");
+  jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, thr,
+          "java/lang/Thread", "getId", "()J");
   if (jthr) {
     snprintf(id, max, "%s", UNKNOWN);
     printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -117,8 +117,8 @@ static void get_current_thread_id(JNIEnv* env, char* id, int max) {
   }
   thr_id = jVal.j;
 
-  jthr = invokeMethod(env, &jVal, INSTANCE, thr, "java/lang/Thread",
-                      "toString", "()Ljava/lang/String;");
+  jthr = findClassAndInvokeMethod(env, &jVal, INSTANCE, thr,
+          "java/lang/Thread", "toString", "()Ljava/lang/String;");
   if (jthr) {
     snprintf(id, max, "%s:%ld", UNKNOWN, thr_id);
     printExceptionAndFree(env, jthr, PRINT_EXC_ALL,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/mutexes.c

@@ -20,8 +20,8 @@
 
 #include <windows.h>
 
-mutex hdfsHashMutex;
 mutex jvmMutex;
+mutex jclassInitMutex;
 
 /**
  * Unfortunately, there is no simple static initializer for a critical section.
@@ -34,8 +34,8 @@ mutex jvmMutex;
  * http://msdn.microsoft.com/en-us/library/bb918180.aspx
  */
 static void __cdecl initializeMutexes(void) {
-  InitializeCriticalSection(&hdfsHashMutex);
   InitializeCriticalSection(&jvmMutex);
+  InitializeCriticalSection(&jclassInitMutex);
 }
 #pragma section(".CRT$XCU", read)
 __declspec(allocate(".CRT$XCU"))

この差分においてかなりの量のファイルが変更されているため、一部のファイルを表示していません