You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by jh...@apache.org on 2016/12/11 15:53:07 UTC

hadoop git commit: HDFS-10796: libhdfs++: Previous commit was missing test/hdfs_ioservice_test.cc, adding it.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-8707 e7f4e6679 -> 2ceec2b3c


HDFS-10796: libhdfs++: Previous commit was missing test/hdfs_ioservice_test.cc, adding it.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2ceec2b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2ceec2b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2ceec2b3

Branch: refs/heads/HDFS-8707
Commit: 2ceec2b3c4767fd4b61ffa4656760dbf01465a2b
Parents: e7f4e66
Author: James <jh...@apache.org>
Authored: Sun Dec 11 10:52:39 2016 -0500
Committer: James <jh...@apache.org>
Committed: Sun Dec 11 10:52:39 2016 -0500

----------------------------------------------------------------------
 .../libhdfspp/tests/hdfs_ioservice_test.cc      | 103 +++++++++++++++++++
 1 file changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ceec2b3/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc
new file mode 100644
index 0000000..a03f275
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc
@@ -0,0 +1,103 @@
+/**
+ * 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/hdfs_ioservice.h"
+
+#include <future>
+#include <functional>
+#include <thread>
+#include <string>
+
+#include <gmock/gmock.h>
+
+using ::testing::_;
+using ::testing::InvokeArgument;
+using ::testing::Return;
+
+using namespace hdfs;
+
+// Make sure IoService spins up specified number of threads
+TEST(IoServiceTest, InitThreads) {
+#ifndef DISABLE_CONCURRENT_WORKERS
+  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  EXPECT_NE(service, nullptr);
+
+  unsigned int thread_count = 4;
+  unsigned int result_thread_count = service->InitWorkers(thread_count);
+  EXPECT_EQ(thread_count, result_thread_count);
+
+  service->Stop();
+#else
+  #pragma message("DISABLE_CONCURRENT_WORKERS is defined so hdfs_ioservice_test will compile out the InitThreads test")
+#endif
+}
+
+// Make sure IoService defaults to logical thread count
+TEST(IoServiceTest, InitDefaultThreads) {
+#ifndef DISABLE_CONCURRENT_WORKERS
+  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  EXPECT_NE(service, nullptr);
+
+  unsigned int thread_count = std::thread::hardware_concurrency();
+  unsigned int result_thread_count = service->InitDefaultWorkers();
+  EXPECT_EQ(thread_count, result_thread_count);
+
+  service->Stop();
+#else
+  #pragma message("DISABLE_CONCURRENT_WORKERS is defined so hdfs_ioservice_test will compile out the InitDefaultThreads test")
+#endif
+}
+
+
+// Check IoService::PostTask
+TEST(IoServiceTest, SimplePost) {
+  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  EXPECT_NE(service, nullptr);
+
+  unsigned int thread_count = std::thread::hardware_concurrency();
+  unsigned int result_thread_count = service->InitDefaultWorkers();
+#ifndef DISABLE_CONCURRENT_WORKERS
+  EXPECT_EQ(thread_count, result_thread_count);
+#else
+  (void)thread_count;
+  (void)result_thread_count;
+#endif
+  // Like with the C synchronous shims a promise/future is needed to block until the async call completes.
+  auto promise = std::make_shared<std::promise<std::string>>();
+  std::future<std::string> future = promise->get_future();
+
+  // this will get invoked on a worker thread
+  std::function<void()> example_callback = [promise](){
+    promise->set_value("hello from IoService");
+  };
+  service->PostTask(example_callback);
+
+  // block until worker thread finishes
+  std::string result = future.get();
+  EXPECT_EQ(result, "hello from IoService");
+
+  service->Stop();
+
+}
+
+int main(int argc, char *argv[]) {
+  // The following line must be executed to initialize Google Mock
+  // (and Google Test) before running the tests.
+  ::testing::InitGoogleMock(&argc, argv);
+  return RUN_ALL_TESTS();
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org