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 in...@apache.org on 2021/10/13 16:11:54 UTC

[hadoop] branch trunk updated: HDFS-16267. Make hdfs_df tool cross platform (#3542)

This is an automated email from the ASF dual-hosted git repository.

inigoiri pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 107fe22  HDFS-16267. Make hdfs_df tool cross platform (#3542)
107fe22 is described below

commit 107fe227eb0f9694cf316da8f0ef56654909ddd0
Author: Gautham B A <ga...@gmail.com>
AuthorDate: Wed Oct 13 21:41:21 2021 +0530

    HDFS-16267. Make hdfs_df tool cross platform (#3542)
---
 .../native/libhdfspp/tests/tools/CMakeLists.txt    |   3 +
 .../native/libhdfspp/tests/tools/hdfs-df-mock.cc   |  53 ++++++++++
 .../native/libhdfspp/tests/tools/hdfs-df-mock.h    |  67 ++++++++++++
 .../libhdfspp/tests/tools/hdfs-tool-tests.cc       |   9 ++
 .../src/main/native/libhdfspp/tools/CMakeLists.txt |   3 +-
 .../native/libhdfspp/tools/hdfs-df/CMakeLists.txt  |  27 +++++
 .../main/native/libhdfspp/tools/hdfs-df/hdfs-df.cc | 113 +++++++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs-df/hdfs-df.h  |  90 ++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs-df/main.cc    |  52 ++++++++++
 .../src/main/native/libhdfspp/tools/hdfs_df.cc     |  89 ----------------
 10 files changed, 415 insertions(+), 91 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt
index 023b225..48580a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt
@@ -22,15 +22,18 @@ add_executable(hdfs_tool_tests
                 hdfs-cat-mock.cc
                 hdfs-tool-test-fixtures.cc
                 hdfs-tool-tests.cc
+                hdfs-df-mock.cc
                 main.cc)
 target_include_directories(hdfs_tool_tests PRIVATE
                             ../tools
                             ../../tools
+                            ../../tools/hdfs-df
                             ../../tools/hdfs-allow-snapshot
                             ../../tools/hdfs-delete-snapshot
                             ../../tools/hdfs-cat)
 target_link_libraries(hdfs_tool_tests PRIVATE
                        gmock_main
+                       hdfs_df_lib
                        hdfs_allowSnapshot_lib
                        hdfs_deleteSnapshot_lib
                        hdfs_cat_lib)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.cc
new file mode 100644
index 0000000..301fddd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.cc
@@ -0,0 +1,53 @@
+/**
+ * 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 <functional>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "hdfs-df-mock.h"
+#include "hdfs-tool-tests.h"
+
+namespace hdfs::tools::test {
+DfMock::~DfMock() = default;
+
+void DfMock::SetExpectations(std::function<std::unique_ptr<DfMock>()> test_case,
+                             const std::vector<std::string> &args) const {
+  // Get the pointer to the function that defines the test case
+  const auto test_case_func = test_case.target<std::unique_ptr<DfMock> (*)()>();
+  ASSERT_NE(test_case_func, nullptr);
+
+  // Set the expected method calls and their corresponding arguments for each
+  // test case
+  if (*test_case_func == &CallHelp<DfMock>) {
+    EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true));
+    return;
+  }
+
+  if (*test_case_func == &PassAPath<DfMock>) {
+    const auto arg1 = args[0];
+    EXPECT_CALL(*this, HandlePath(arg1))
+        .Times(1)
+        .WillOnce(testing::Return(true));
+  }
+}
+} // namespace hdfs::tools::test
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.h
new file mode 100644
index 0000000..3722106
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-df-mock.h
@@ -0,0 +1,67 @@
+/**
+ * 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 LIBHDFSPP_TOOLS_HDFS_DF_MOCK
+#define LIBHDFSPP_TOOLS_HDFS_DF_MOCK
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+
+#include "hdfs-df.h"
+
+namespace hdfs::tools::test {
+/**
+ * {@class DfMock} is an {@class Df} whereby it mocks the
+ * HandleHelp and HandlePath methods for testing their functionality.
+ */
+class DfMock : public hdfs::tools::Df {
+public:
+  /**
+   * {@inheritdoc}
+   */
+  DfMock(const int argc, char **argv) : Df(argc, argv) {}
+
+  // Abiding to the Rule of 5
+  DfMock(const DfMock &) = delete;
+  DfMock(DfMock &&) = delete;
+  DfMock &operator=(const DfMock &) = delete;
+  DfMock &operator=(DfMock &&) = delete;
+  ~DfMock() override;
+
+  /**
+   * Defines the methods and the corresponding arguments that are expected
+   * to be called on this instance of {@link HdfsTool} for the given test case.
+   *
+   * @param test_case An {@link std::function} object that points to the
+   * function defining the test case
+   * @param args The arguments that are passed to this test case
+   */
+  void SetExpectations(std::function<std::unique_ptr<DfMock>()> test_case,
+                       const std::vector<std::string> &args = {}) const;
+
+  MOCK_METHOD(bool, HandleHelp, (), (const, override));
+
+  MOCK_METHOD(bool, HandlePath, (const std::string &), (const, override));
+};
+} // namespace hdfs::tools::test
+
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc
index 9641fbf..8ffce3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc
@@ -20,6 +20,7 @@
 #include <gtest/gtest.h>
 
 #include "hdfs-allow-snapshot-mock.h"
+#include "hdfs-df-mock.h"
 #include "hdfs-cat-mock.h"
 #include "hdfs-delete-snapshot-mock.h"
 #include "hdfs-tool-test-fixtures.h"
@@ -46,6 +47,10 @@ INSTANTIATE_TEST_SUITE_P(HdfsCat, HdfsToolBasicTest,
                          testing::Values(PassAPath<hdfs::tools::test::CatMock>,
                                          CallHelp<hdfs::tools::test::CatMock>));
 
+INSTANTIATE_TEST_SUITE_P(HdfsDf, HdfsToolBasicTest,
+                         testing::Values(PassAPath<hdfs::tools::test::DfMock>,
+                                         CallHelp<hdfs::tools::test::DfMock>));
+
 INSTANTIATE_TEST_SUITE_P(
     HdfsDeleteSnapshot, HdfsToolBasicTest,
     testing::Values(CallHelp<hdfs::tools::test::DeleteSnapshotMock>,
@@ -56,6 +61,10 @@ INSTANTIATE_TEST_SUITE_P(
     testing::Values(Pass2Paths<hdfs::tools::test::AllowSnapshotMock>));
 
 INSTANTIATE_TEST_SUITE_P(
+    HdfsDf, HdfsToolNegativeTestThrows,
+    testing::Values(Pass2Paths<hdfs::tools::test::DfMock>));
+
+INSTANTIATE_TEST_SUITE_P(
     HdfsCat, HdfsToolNegativeTestThrows,
     testing::Values(Pass2Paths<hdfs::tools::test::CatMock>));
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt
index 8be19a9..c122250 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt
@@ -60,8 +60,7 @@ target_link_libraries(hdfs_stat tools_common hdfspp_static)
 add_executable(hdfs_count hdfs_count.cc)
 target_link_libraries(hdfs_count tools_common hdfspp_static)
 
-add_executable(hdfs_df hdfs_df.cc)
-target_link_libraries(hdfs_df tools_common hdfspp_static)
+add_subdirectory(hdfs-df)
 
 add_executable(hdfs_du hdfs_du.cc)
 target_link_libraries(hdfs_du tools_common hdfspp_static)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/CMakeLists.txt
new file mode 100644
index 0000000..6936c9f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/CMakeLists.txt
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+add_library(hdfs_df_lib STATIC $<TARGET_OBJECTS:hdfs_tool_obj> hdfs-df.cc)
+target_include_directories(hdfs_df_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS})
+target_link_libraries(hdfs_df_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static)
+
+add_executable(hdfs_df main.cc)
+target_include_directories(hdfs_df PRIVATE ../../tools)
+target_link_libraries(hdfs_df PRIVATE hdfs_df_lib)
+
+install(TARGETS hdfs_df RUNTIME DESTINATION bin)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.cc
new file mode 100644
index 0000000..ec832ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.cc
@@ -0,0 +1,113 @@
+/**
+ * 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 <iostream>
+#include <memory>
+#include <ostream>
+#include <sstream>
+#include <string>
+
+#include "hdfs-df.h"
+#include "tools_common.h"
+
+namespace hdfs::tools {
+Df::Df(const int argc, char **argv) : HdfsTool(argc, argv) {}
+
+bool Df::Initialize() {
+  auto add_options = opt_desc_.add_options();
+  add_options("help,h", "Displays size, used space, and available space of "
+                        "the entire filesystem where PATH is located");
+  add_options("path", po::value<std::string>(),
+              "The path indicating the filesystem that needs to be df-ed");
+
+  // We allow only one argument to be passed to this tool. An exception is
+  // thrown if multiple arguments are passed.
+  pos_opt_desc_.add("path", 1);
+
+  po::store(po::command_line_parser(argc_, argv_)
+                .options(opt_desc_)
+                .positional(pos_opt_desc_)
+                .run(),
+            opt_val_);
+  po::notify(opt_val_);
+  return true;
+}
+
+std::string Df::GetDescription() const {
+  std::stringstream desc;
+  desc << "Usage: hdfs_df [OPTION] PATH" << std::endl
+       << std::endl
+       << "Displays size, used space, and available space of" << std::endl
+       << "the entire filesystem where PATH is located" << std::endl
+       << std::endl
+       << "  -h        display this help and exit" << std::endl
+       << std::endl
+       << "Examples:" << std::endl
+       << "hdfs_df hdfs://localhost.localdomain:8020/" << std::endl
+       << "hdfs_df /" << std::endl;
+  return desc.str();
+}
+
+bool Df::Do() {
+  if (!Initialize()) {
+    std::cerr << "Unable to initialize HDFS df tool" << std::endl;
+    return false;
+  }
+
+  if (!ValidateConstraints()) {
+    std::cout << GetDescription();
+    return false;
+  }
+
+  if (opt_val_.count("help") > 0) {
+    return HandleHelp();
+  }
+
+  if (opt_val_.count("path") > 0) {
+    const auto path = opt_val_["path"].as<std::string>();
+    return HandlePath(path);
+  }
+
+  return true;
+}
+
+bool Df::HandleHelp() const {
+  std::cout << GetDescription();
+  return true;
+}
+
+bool Df::HandlePath(const std::string &path) const {
+  // Building a URI object from the given uri_path
+  auto uri = hdfs::parse_path_or_exit(path);
+
+  const auto fs = hdfs::doConnect(uri, false);
+  if (!fs) {
+    std::cerr << "Error: Could not connect the file system." << std::endl;
+    return false;
+  }
+
+  hdfs::FsInfo fs_info;
+  const auto status = fs->GetFsStats(fs_info);
+  if (!status.ok()) {
+    std::cerr << "Error: " << status.ToString() << std::endl;
+    return false;
+  }
+  std::cout << fs_info.str("hdfs://" + fs->get_cluster_name()) << std::endl;
+  return true;
+}
+} // namespace hdfs::tools
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.h
new file mode 100644
index 0000000..27cdc9f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/hdfs-df.h
@@ -0,0 +1,90 @@
+/**
+ * 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 LIBHDFSPP_TOOLS_HDFS_DF
+#define LIBHDFSPP_TOOLS_HDFS_DF
+
+#include <string>
+
+#include <boost/program_options.hpp>
+
+#include "hdfs-tool.h"
+
+namespace hdfs::tools {
+/**
+ * {@class Df} is an {@class HdfsTool} that displays size, used space, and
+ * available space of the entire filesystem where the given path is located.
+ */
+class Df : public HdfsTool {
+public:
+  /**
+   * {@inheritdoc}
+   */
+  Df(int argc, char **argv);
+
+  // Abiding to the Rule of 5
+  Df(const Df &) = default;
+  Df(Df &&) = default;
+  Df &operator=(const Df &) = delete;
+  Df &operator=(Df &&) = delete;
+  ~Df() override = default;
+
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] std::string GetDescription() const override;
+
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] bool Do() override;
+
+protected:
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] bool Initialize() override;
+
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] bool ValidateConstraints() const override { return argc_ > 1; }
+
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] bool HandleHelp() const override;
+
+  /**
+   * Handle the path argument that's passed to this tool.
+   *
+   * @param path The path to the directory for which we need df info.
+   *
+   * @return A boolean indicating the result of this operation.
+   */
+  [[nodiscard]] virtual bool HandlePath(const std::string &path) const;
+
+private:
+  /**
+   * A boost data-structure containing the description of positional arguments
+   * passed to the command-line.
+   */
+  po::positional_options_description pos_opt_desc_;
+};
+} // namespace hdfs::tools
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/main.cc
new file mode 100644
index 0000000..cced290
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-df/main.cc
@@ -0,0 +1,52 @@
+/**
+ * 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 <cstdlib>
+#include <exception>
+#include <iostream>
+
+#include <google/protobuf/stubs/common.h>
+
+#include "hdfs-df.h"
+
+int main(int argc, char *argv[]) {
+  const auto result = std::atexit([]() -> void {
+    // Clean up static data on exit and prevent valgrind memory leaks
+    google::protobuf::ShutdownProtobufLibrary();
+  });
+  if (result != 0) {
+    std::cerr
+        << "Error: Unable to schedule clean-up tasks for HDFS df tool, exiting"
+        << std::endl;
+    std::exit(EXIT_FAILURE);
+  }
+
+  hdfs::tools::Df df(argc, argv);
+  auto success = false;
+
+  try {
+    success = df.Do();
+  } catch (const std::exception &e) {
+    std::cerr << "Error: " << e.what() << std::endl;
+  }
+
+  if (!success) {
+    std::exit(EXIT_FAILURE);
+  }
+  return 0;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_df.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_df.cc
deleted file mode 100644
index dddf010..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_df.cc
+++ /dev/null
@@ -1,89 +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 <google/protobuf/stubs/common.h>
-#include <unistd.h>
-#include "tools_common.h"
-
-void usage(){
-  std::cout << "Usage: hdfs_df [OPTION] PATH"
-      << std::endl
-      << std::endl << "Displays size, used space, and available space of"
-      << std::endl << "the entire filesystem where PATH is located"
-      << std::endl
-      << std::endl << "  -h        display this help and exit"
-      << std::endl
-      << std::endl << "Examples:"
-      << std::endl << "hdfs_df hdfs://localhost.localdomain:8020/"
-      << std::endl << "hdfs_df /"
-      << std::endl;
-}
-
-int main(int argc, char *argv[]) {
-  //We should have at least 2 arguments
-  if (argc < 2) {
-    usage();
-    exit(EXIT_FAILURE);
-  }
-
-  int input;
-
-  //Using GetOpt to read in the values
-  opterr = 0;
-  while ((input = getopt(argc, argv, "h")) != -1) {
-    switch (input)
-    {
-    case 'h':
-      usage();
-      exit(EXIT_SUCCESS);
-    case '?':
-      if (isprint(optopt))
-        std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl;
-      else
-        std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl;
-      usage();
-      exit(EXIT_FAILURE);
-    default:
-      exit(EXIT_FAILURE);
-    }
-  }
-  std::string uri_path = argv[optind];
-
-  //Building a URI object from the given uri_path
-  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
-
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
-  if (!fs) {
-    std::cerr << "Could not connect the file system. " << std::endl;
-    exit(EXIT_FAILURE);
-  }
-
-  hdfs::FsInfo fs_info;
-
-  hdfs::Status status = fs->GetFsStats(fs_info);
-  if (!status.ok()) {
-    std::cerr << "Error: " << status.ToString() << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  std::cout << fs_info.str("hdfs://" + fs->get_cluster_name()) << std::endl;
-
-  // Clean up static data and prevent valgrind memory leaks
-  google::protobuf::ShutdownProtobufLibrary();
-  return 0;
-}

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