Pārlūkot izejas kodu

HDFS-7807. libhdfs htable.c: fix htable resizing, add unit test (cmccabe)

Colin Patrick Mccabe 10 gadi atpakaļ
vecāks
revīzija
585768667e

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1009,6 +1009,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7009. Active NN and standby NN have different live nodes.
     (Ming Ma via cnauroth)
 
+    HDFS-7807. libhdfs htable.c: fix htable resizing, add unit test (cmccabe)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -211,6 +211,14 @@ target_link_libraries(test_libhdfs_zerocopy
     ${OS_LINK_LIBRARIES}
 )
 
+add_executable(test_htable
+    main/native/libhdfs/common/htable.c
+    main/native/libhdfs/test/test_htable.c
+)
+target_link_libraries(test_htable
+    ${OS_LINK_LIBRARIES}
+)
+
 # Skip vecsum on Windows.  This could be made to work in the future by
 # introducing an abstraction layer over the sys/mman.h functions.
 if (NOT WIN32)

+ 20 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/common/htable.c

@@ -83,8 +83,10 @@ static int htable_realloc(struct htable *htable, uint32_t new_capacity)
     }
     for (i = 0; i < old_capacity; i++) {
         struct htable_pair *pair = htable->elem + i;
-        htable_insert_internal(nelem, new_capacity, hash_fun,
-                               pair->key, pair->val);
+        if (pair->key) {
+            htable_insert_internal(nelem, new_capacity, hash_fun,
+                                   pair->key, pair->val);
+        }
     }
     free(htable->elem);
     htable->elem = nelem;
@@ -92,6 +94,21 @@ static int htable_realloc(struct htable *htable, uint32_t 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)
 {
@@ -101,8 +118,7 @@ struct htable *htable_alloc(uint32_t size,
     if (!htable) {
         return NULL;
     }
-    size = (size + 1) >> 1;
-    size = size << 1;
+    size = round_up_to_power_of_2(size);
     if (size < HTABLE_MIN_SIZE) {
         size = HTABLE_MIN_SIZE;
     }

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

@@ -0,0 +1,100 @@
+/**
+ * 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