瀏覽代碼

HADOOP-10636. Add unit test case for call and client_id (Wenwu Peng via Binglin Chang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HADOOP-10388@1603351 13f79535-47bb-0310-9956-ffa450edef68
Binglin Chang 11 年之前
父節點
當前提交
928e93f5b7

+ 10 - 0
hadoop-native-core/src/main/native/CMakeLists.txt

@@ -177,6 +177,16 @@ add_executable(uri-unit common/uri-unit.c
 add_utest(uri-unit)
 target_link_libraries(uri-unit ${URIPARSER_LIB} ${LIBUV_LIB})
 
+add_executable(client_id-unit rpc/client_id-unit.c
+    rpc/client_id.c common/hadoop_err.c test/test.c)
+add_utest(client_id-unit)
+target_link_libraries(client_id-unit fstest)
+
+add_executable(call-unit rpc/call-unit.c
+    rpc/call.c common/hadoop_err.c test/test.c)
+add_utest(call-unit)
+target_link_libraries(call-unit fstest)
+
 add_executable(namenode-rpc-unit
     ndfs/namenode-rpc-unit.c)
 target_link_libraries(namenode-rpc-unit fstest)

+ 3 - 0
hadoop-native-core/src/main/native/common/test.h

@@ -149,6 +149,9 @@ void *xcalloc(size_t len);
     EXPECT("0", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_NE, \
            "%"PRIdPTR, x);
 
+#define EXPECT_NULL(x) \
+    EXPECT("0", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+           "%"PRIdPTR, x);
 #endif
 
 // vim: ts=4:sw=4:tw=79:et

+ 142 - 0
hadoop-native-core/src/main/native/rpc/call-unit.c

@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ #include "rpc/call.h"
+ #include "common/test.h"
+ #include "common/hadoop_err.h"
+ #include "rpc/proxy.h"
+
+ #include <stdlib.h>
+ #include <stdio.h>
+ #include <uv.h>
+ #include <string.h>
+ #include <stdint.h>
+
+int err_test(struct hrpc_response *resp, struct hadoop_err *err)
+{
+    EXPECT_STR_EQ("org.apache.hadoop.native.HadoopCore.InvalidRequestException: "
+                        "test hadoop_lerr_alloc", hadoop_err_msg(err));
+    EXPECT_NULL(resp);
+    hadoop_err_free(err);
+    return 0;
+}
+
+void hrpc_call_deliver_err_test_cb(struct hrpc_response *resp,
+                            struct hadoop_err *err, void *cb_data)
+{
+    uv_sem_t *sem = cb_data;
+    if (err_test(resp, err))
+    {
+        abort();
+    }
+    uv_sem_post(sem);
+}
+
+int hrpc_call_deliver_err_test(struct hrpc_call *call, struct hadoop_err *err)
+{
+    hrpc_call_deliver_err(call, err);
+    return 0;
+}
+
+int resp_test(struct hrpc_response *resp, struct hadoop_err *err)
+{
+    EXPECT_NONNULL(resp);
+    EXPECT_NULL(err);
+    free(resp->base);
+    return 0;
+}
+
+void hrpc_call_deliver_resp_test_cb(struct hrpc_response *resp,
+                            struct hadoop_err *err, void *cb_data)
+{
+    uv_sem_t *sem = cb_data;
+    if (resp_test(resp, err))
+    {
+        abort();
+    }
+    uv_sem_post(sem);
+}
+
+int hrpc_call_deliver_resp_test(struct hrpc_call *call, struct hrpc_response *resp)
+{
+     hrpc_call_deliver_resp(call, resp);
+     return 0;
+}
+
+int hrpc_call_test(struct hrpc_call *call)
+{
+     // Test hrpc_call_activate
+    EXPECT_INT_EQ(1, hrpc_call_activate(call));
+    EXPECT_INT_EQ(1, hrpc_call_is_active(call));
+
+    // Test hrpc_call_activate if it was already active
+    EXPECT_INT_EQ(0, hrpc_call_activate(call));
+    EXPECT_INT_EQ(1, hrpc_call_is_active(call));
+
+    //Test hrpc_call_deactivate
+    hrpc_call_deactivate(call);
+    EXPECT_INT_EQ(0, hrpc_call_is_active(call));
+
+    return 0;
+}
+
+int main(void)
+{
+    struct hrpc_call call;
+    struct hadoop_err *err;
+    struct hrpc_response resp;
+    uv_buf_t payload;
+    uv_sem_t sem1, sem2;
+
+    EXPECT_INT_ZERO(uv_sem_init(&sem1, 0));
+    EXPECT_INT_ZERO(uv_sem_init(&sem2, 0));
+
+    payload.base = strdup("testbuff");
+    payload.len = strlen(payload.base);
+
+    call.remote.sin_addr.s_addr = inet_addr("127.0.0.1");
+    call.protocol = "org.apache.hadoop.hdfs.protocol.ClientProtocol";
+    call.username = "root";
+    call.payload = payload;
+    call.cb = hrpc_call_deliver_err_test_cb;
+    call.cb_data = &sem1;
+    call.active = 0;
+    err =  hadoop_lerr_alloc(EINVAL, "test hadoop_lerr_alloc");
+
+    //Test hrpc call
+    EXPECT_INT_ZERO(hrpc_call_test(&call));
+
+    //Test hrpc call deliver error
+    EXPECT_INT_ZERO(hrpc_call_deliver_err_test(&call, err));
+    uv_sem_wait(&sem1);
+
+    call.cb_data = &sem2;
+    call.cb = hrpc_call_deliver_resp_test_cb;
+
+    resp.pb_base = (uint8_t*)(payload.len);
+    resp.pb_len = (int32_t)(payload.len);
+    resp.base = strdup("testbuff");
+
+    //Test hrpc call deliver response
+    EXPECT_INT_ZERO(hrpc_call_deliver_resp_test(&call, &resp));
+    uv_sem_wait(&sem2);
+
+    return EXIT_SUCCESS;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 64 - 0
hadoop-native-core/src/main/native/rpc/client_id-unit.c

@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "rpc/client_id.h"
+#include "common/test.h"
+#include "common/hadoop_err.h"
+
+#include <string.h>
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <uv.h>
+
+#define HRPC_CLIENT_ID_LEN 16
+
+int test_hrpc_client_id(void)
+{
+    struct hrpc_client_id id, id1;
+    char str[36];
+    char bytes[16] = { 0 };
+    struct hadoop_err *err;
+
+    memset(&id, 0, sizeof(id));
+    memset(&id1, 0, sizeof(id1));
+
+    hrpc_client_id_generate_rand(&id);
+    hrpc_client_id_generate_rand(&id1);
+
+    EXPECT_INT_ZERO(!hrpc_client_id_compare(&id, &id1));
+    EXPECT_INT_ZERO(hrpc_client_id_compare(&id, &id));
+
+    EXPECT_NO_HADOOP_ERR(hrpc_client_id_from_bytes(&bytes, HRPC_CLIENT_ID_LEN, &id));
+    EXPECT_STR_EQ("00000000-00000000-00000000-00000000"
+                    ,hrpc_client_id_to_str(&id, str, sizeof(str)));
+
+    err = hrpc_client_id_from_bytes(&bytes, 6, &id);
+    EXPECT_STR_EQ("org.apache.hadoop.native.HadoopCore.InvalidRequestException: "
+                    "hrpc_client_id_from_bytes: "
+                    "invalid client id length of 6 (expected 16)",
+                    hadoop_err_msg(err));
+    hadoop_err_free(err);
+    return 0;
+}
+
+int main(void) {
+    EXPECT_INT_ZERO(test_hrpc_client_id());
+    return EXIT_SUCCESS;
+}
+  // vim: ts=4:sw=4:tw=79:et