Ver código fonte

HDFS-16474. Make HDFS tail tool cross platform (#4157)

* The source files for hdfs_tail
  uses getopt for parsing the
  command line arguments.
* getopt is available only on
  Linux and thus, isn't cross
  platform.
* We need to replace getopt
  with boost::program_options
  to make these tools cross
  platform.
Gautham B A 3 anos atrás
pai
commit
e89fd9645b

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt

@@ -40,6 +40,7 @@ add_executable(hdfs_tool_tests
                 hdfs-ls-mock.cc
                 hdfs-setrep-mock.cc
                 hdfs-stat-mock.cc
+                hdfs-tail-mock.cc
                 main.cc)
 target_include_directories(hdfs_tool_tests PRIVATE
                             ../tools
@@ -64,6 +65,7 @@ target_include_directories(hdfs_tool_tests PRIVATE
                             ../../tools/hdfs-ls
                             ../../tools/hdfs-setrep
                             ../../tools/hdfs-stat
+                            ../../tools/hdfs-tail
                             ../../tools/hdfs-cat)
 target_link_libraries(hdfs_tool_tests PRIVATE
                        gmock_main
@@ -87,5 +89,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE
                        hdfs_ls_lib
                        hdfs_setrep_lib
                        hdfs_stat_lib
+                       hdfs_tail_lib
                        hdfs_cat_lib)
 add_test(hdfs_tool_tests hdfs_tool_tests)

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.cc

@@ -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 <functional>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "hdfs-tail-mock.h"
+#include "hdfs-tool-tests.h"
+
+namespace hdfs::tools::test {
+TailMock::~TailMock() = default;
+
+void TailMock::SetExpectations(
+    std::function<std::unique_ptr<TailMock>()> 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<TailMock> (*)()>();
+  ASSERT_NE(test_case_func, nullptr);
+
+  // Set the expected method calls and their corresponding arguments for each
+  // test case
+  if (*test_case_func == &CallHelp<TailMock>) {
+    EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true));
+    return;
+  }
+
+  if (*test_case_func == &PassAPath<TailMock>) {
+    const auto path = args[0];
+    EXPECT_CALL(*this, HandlePath(path, false))
+        .Times(1)
+        .WillOnce(testing::Return(true));
+  }
+
+  if (*test_case_func == &PassFOptAndAPath<TailMock>) {
+    const auto f_opt = args[0];
+    const auto path = args[1];
+    ASSERT_EQ(f_opt, "-f");
+    EXPECT_CALL(*this, HandlePath(path, true))
+        .Times(1)
+        .WillOnce(testing::Return(true));
+  }
+}
+} // namespace hdfs::tools::test

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.h

@@ -0,0 +1,68 @@
+/**
+ * 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_TAIL_MOCK
+#define LIBHDFSPP_TOOLS_HDFS_TAIL_MOCK
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+
+#include "hdfs-tail.h"
+
+namespace hdfs::tools::test {
+/**
+ * {@class TailMock} is an {@class Tail} whereby it mocks the
+ * HandleHelp and HandlePath methods for testing their functionality.
+ */
+class TailMock : public hdfs::tools::Tail {
+public:
+  /**
+   * {@inheritdoc}
+   */
+  TailMock(const int argc, char **argv) : Tail(argc, argv) {}
+
+  // Abiding to the Rule of 5
+  TailMock(const TailMock &) = delete;
+  TailMock(TailMock &&) = delete;
+  TailMock &operator=(const TailMock &) = delete;
+  TailMock &operator=(TailMock &&) = delete;
+  ~TailMock() 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<TailMock>()> test_case,
+                       const std::vector<std::string> &args = {}) const;
+
+  MOCK_METHOD(bool, HandleHelp, (), (const, override));
+
+  MOCK_METHOD(bool, HandlePath, (const std::string &, const bool),
+              (const, override));
+};
+} // namespace hdfs::tools::test
+
+#endif

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc

@@ -40,6 +40,7 @@
 #include "hdfs-rm-mock.h"
 #include "hdfs-setrep-mock.h"
 #include "hdfs-stat-mock.h"
+#include "hdfs-tail-mock.h"
 #include "hdfs-tool-test-fixtures.h"
 #include "hdfs-tool-tests.h"
 
@@ -168,6 +169,12 @@ INSTANTIATE_TEST_SUITE_P(
     testing::Values(CallHelp<hdfs::tools::test::StatMock>,
                     PassAPath<hdfs::tools::test::StatMock>));
 
+INSTANTIATE_TEST_SUITE_P(
+    HdfsTail, HdfsToolBasicTest,
+    testing::Values(PassAPath<hdfs::tools::test::TailMock>,
+                    CallHelp<hdfs::tools::test::TailMock>,
+                    PassFOptAndAPath<hdfs::tools::test::TailMock>));
+
 // Negative tests
 INSTANTIATE_TEST_SUITE_P(
     HdfsAllowSnapshot, HdfsToolNegativeTestThrows,
@@ -282,6 +289,20 @@ INSTANTIATE_TEST_SUITE_P(
                     PassMOpt<hdfs::tools::test::StatMock>,
                     PassNOpt<hdfs::tools::test::StatMock>));
 
+INSTANTIATE_TEST_SUITE_P(
+    HdfsTail, HdfsToolNegativeTestThrows,
+    testing::Values(Pass2Paths<hdfs::tools::test::TailMock>,
+                    Pass3Paths<hdfs::tools::test::TailMock>,
+                    PassNOptAndAPath<hdfs::tools::test::TailMock>,
+                    PassRecursiveOwnerAndAPath<hdfs::tools::test::TailMock>,
+                    PassMOpt<hdfs::tools::test::TailMock>,
+                    PassRecursive<hdfs::tools::test::TailMock>,
+                    PassRecursivePath<hdfs::tools::test::TailMock>,
+                    PassNOpt<hdfs::tools::test::TailMock>,
+                    PassOwnerAndAPath<hdfs::tools::test::TailMock>,
+                    PassMPOptsPermissionsAndAPath<hdfs::tools::test::TailMock>,
+                    PassPermissionsAndAPath<hdfs::tools::test::TailMock>));
+
 INSTANTIATE_TEST_SUITE_P(
     HdfsRm, HdfsToolNegativeTestNoThrow,
     testing::Values(PassRecursive<hdfs::tools::test::RmMock>));
@@ -341,3 +362,7 @@ INSTANTIATE_TEST_SUITE_P(
 INSTANTIATE_TEST_SUITE_P(
     HdfsSetrep, HdfsToolNegativeTestNoThrow,
     testing::Values(PassAPath<hdfs::tools::test::SetrepMock>));
+
+INSTANTIATE_TEST_SUITE_P(
+    HdfsTail, HdfsToolNegativeTestNoThrow,
+    testing::Values(PassFOpt<hdfs::tools::test::TailMock>));

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h

@@ -69,6 +69,19 @@ template <class T> std::unique_ptr<T> PassRecursivePath() {
   return hdfs_tool;
 }
 
+template <class T> std::unique_ptr<T> PassFOptAndAPath() {
+  constexpr auto argc = 3;
+  static std::string exe("hdfs_tool_name");
+  static std::string arg1("-f");
+  static std::string arg2("a/b/c");
+
+  static char *argv[] = {exe.data(), arg1.data(), arg2.data()};
+
+  auto hdfs_tool = std::make_unique<T>(argc, argv);
+  hdfs_tool->SetExpectations(PassFOptAndAPath<T>, {arg1, arg2});
+  return hdfs_tool;
+}
+
 template <class T> std::unique_ptr<T> CallHelp() {
   constexpr auto argc = 2;
   static std::string exe("hdfs_tool_name");
@@ -226,6 +239,18 @@ template <class T> std::unique_ptr<T> PassMOpt() {
   return hdfs_tool;
 }
 
+template <class T> std::unique_ptr<T> PassFOpt() {
+  constexpr auto argc = 2;
+  static std::string exe("hdfs_tool_name");
+  static std::string arg1("-f");
+
+  static char *argv[] = {exe.data(), arg1.data()};
+
+  auto hdfs_tool = std::make_unique<T>(argc, argv);
+  hdfs_tool->SetExpectations(PassFOpt<T>, {arg1});
+  return hdfs_tool;
+}
+
 template <class T> std::unique_ptr<T> PassPOptAndPath() {
   constexpr auto argc = 3;
   static std::string exe("hdfs_tool_name");

+ 1 - 23
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt

@@ -34,46 +34,24 @@ add_library(hdfs_tool_obj OBJECT hdfs-tool.cc)
 target_include_directories(hdfs_tool_obj PRIVATE ../tools)
 
 add_subdirectory(hdfs-cat)
-
 add_subdirectory(hdfs-chgrp)
-
 add_subdirectory(hdfs-chown)
-
 add_subdirectory(hdfs-chmod)
-
 add_subdirectory(hdfs-find)
-
 add_subdirectory(hdfs-mkdir)
-
 add_subdirectory(hdfs-rm)
-
 add_subdirectory(hdfs-ls)
-
 add_subdirectory(hdfs-stat)
-
 add_subdirectory(hdfs-count)
-
 add_subdirectory(hdfs-df)
-
 add_subdirectory(hdfs-du)
-
 add_subdirectory(hdfs-get)
-
 add_subdirectory(hdfs-copy-to-local)
-
 add_subdirectory(hdfs-move-to-local)
-
 add_subdirectory(hdfs-setrep)
-
 add_subdirectory(hdfs-allow-snapshot)
-
 add_subdirectory(hdfs-disallow-snapshot)
-
 add_subdirectory(hdfs-create-snapshot)
-
 add_subdirectory(hdfs-rename-snapshot)
-
 add_subdirectory(hdfs-delete-snapshot)
-
-add_executable(hdfs_tail hdfs_tail.cc)
-target_link_libraries(hdfs_tail tools_common hdfspp_static)
+add_subdirectory(hdfs-tail)

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/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_tail_lib STATIC $<TARGET_OBJECTS:hdfs_tool_obj> hdfs-tail.cc)
+target_include_directories(hdfs_tail_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS})
+target_link_libraries(hdfs_tail_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static)
+
+add_executable(hdfs_tail main.cc)
+target_include_directories(hdfs_tail PRIVATE ../../tools)
+target_link_libraries(hdfs_tail PRIVATE hdfs_tail_lib)
+
+install(TARGETS hdfs_tail RUNTIME DESTINATION bin)

+ 150 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.cc

@@ -0,0 +1,150 @@
+/**
+ * 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 <chrono>
+#include <future>
+#include <iostream>
+#include <memory>
+#include <ostream>
+#include <sstream>
+#include <string>
+#include <thread>
+
+#include "hdfs-tail.h"
+#include "tools_common.h"
+
+namespace hdfs::tools {
+Tail::Tail(const int argc, char **argv) : HdfsTool(argc, argv) {}
+
+bool Tail::Initialize() {
+  auto add_options = opt_desc_.add_options();
+  add_options("help,h", "Displays last kilobyte of the file to stdout.");
+  add_options("follow,f",
+              "Append data to the output as the file grows, as in Unix.");
+  add_options("path", po::value<std::string>(),
+              "The path indicating the filesystem that needs to be tailed.");
+
+  // We allow only one positional 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 Tail::GetDescription() const {
+  std::stringstream desc;
+  desc << "Usage: hdfs_tail [OPTION] FILE" << std::endl
+       << std::endl
+       << "Displays last kilobyte of the file to stdout." << std::endl
+       << std::endl
+       << "  -f  append data to the output as the file grows, as in Unix"
+       << std::endl
+       << "  -h  display this help and exit" << std::endl
+       << std::endl
+       << "Examples:" << std::endl
+       << "hdfs_tail hdfs://localhost.localdomain:8020/dir/file" << std::endl
+       << "hdfs_tail /dir/file" << std::endl;
+  return desc.str();
+}
+
+bool Tail::Do() {
+  if (!Initialize()) {
+    std::cerr << "Unable to initialize HDFS tail 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>();
+    const auto follow = opt_val_.count("follow") > 0;
+    return HandlePath(path, follow);
+  }
+
+  return false;
+}
+
+bool Tail::HandleHelp() const {
+  std::cout << GetDescription();
+  return true;
+}
+
+bool Tail::HandlePath(const std::string &path, const bool follow) const {
+  // Building a URI object from the given path.
+  auto uri = hdfs::parse_path_or_exit(path);
+
+  const auto fs = hdfs::doConnect(uri, true);
+  if (!fs) {
+    std::cerr << "Could not connect to the file system." << std::endl;
+    return false;
+  }
+
+  // We need to get the size of the file using stat.
+  hdfs::StatInfo stat_info;
+  auto status = fs->GetFileInfo(uri.get_path(), stat_info);
+  if (!status.ok()) {
+    std::cerr << "Error: " << status.ToString() << std::endl;
+    return false;
+  }
+
+  // Determine where to start reading.
+  off_t offset{0};
+  if (stat_info.length > tail_size_in_bytes) {
+    offset = static_cast<off_t>(stat_info.length - tail_size_in_bytes);
+  }
+
+  do {
+    const auto current_length = static_cast<off_t>(stat_info.length);
+    readFile(fs, uri.get_path(), offset, stdout, false);
+
+    // Exit if -f flag was not set.
+    if (!follow) {
+      break;
+    }
+
+    do {
+      // Sleep for the refresh rate.
+      std::this_thread::sleep_for(std::chrono::seconds(refresh_rate_in_sec));
+
+      // Use stat to check the new file size.
+      status = fs->GetFileInfo(uri.get_path(), stat_info);
+      if (!status.ok()) {
+        std::cerr << "Error: " << status.ToString() << std::endl;
+        return false;
+      }
+
+      // If file became longer, loop back and print the difference.
+    } while (static_cast<off_t>(stat_info.length) <= current_length);
+  } while (true);
+
+  return true;
+}
+} // namespace hdfs::tools

+ 102 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.h

@@ -0,0 +1,102 @@
+/**
+ * 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_TAIL
+#define LIBHDFSPP_TOOLS_HDFS_TAIL
+
+#include <string>
+
+#include <boost/program_options.hpp>
+
+#include "hdfs-tool.h"
+
+namespace hdfs::tools {
+/**
+ * {@class Tail} is an {@class HdfsTool} displays last kilobyte of the file to
+ * stdout.
+ */
+class Tail : public HdfsTool {
+public:
+  /**
+   * {@inheritdoc}
+   */
+  Tail(int argc, char **argv);
+
+  // Abiding to the Rule of 5
+  Tail(const Tail &) = default;
+  Tail(Tail &&) = default;
+  Tail &operator=(const Tail &) = delete;
+  Tail &operator=(Tail &&) = delete;
+  ~Tail() 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 file which needs to be tailed.
+   * @param follow Append data to the output as the file grows, as in Unix.
+   *
+   * @return A boolean indicating the result of this operation.
+   */
+  [[nodiscard]] virtual bool HandlePath(const std::string &path,
+                                        bool follow) const;
+
+  /**
+   * The tail size in bytes.
+   */
+  static constexpr uint64_t tail_size_in_bytes{1024};
+
+  /**
+   * The refresh rate for {@link hdfs::tools::Tail} in seconds.
+   */
+  static constexpr int refresh_rate_in_sec{1};
+
+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

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/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-tail.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 tail tool, "
+                 "exiting"
+              << std::endl;
+    std::exit(EXIT_FAILURE);
+  }
+
+  hdfs::tools::Tail tail(argc, argv);
+  auto success = false;
+
+  try {
+    success = tail.Do();
+  } catch (const std::exception &e) {
+    std::cerr << "Error: " << e.what() << std::endl;
+  }
+
+  if (!success) {
+    std::exit(EXIT_FAILURE);
+  }
+  return 0;
+}

+ 0 - 124
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc

@@ -1,124 +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_tail [OPTION] FILE"
-      << std::endl
-      << std::endl << "Displays last kilobyte of the file to stdout."
-      << std::endl
-      << std::endl << "  -f  output appended data as the file grows, as in Unix"
-      << std::endl << "  -h  display this help and exit"
-      << std::endl
-      << std::endl << "Examples:"
-      << std::endl << "hdfs_tail hdfs://localhost.localdomain:8020/dir/file"
-      << std::endl << "hdfs_tail /dir/file"
-      << std::endl;
-}
-
-#define TAIL_SIZE 1024
-#define REFRESH_RATE 1 //seconds
-
-int main(int argc, char *argv[]) {
-  if (argc < 2) {
-    usage();
-    exit(EXIT_FAILURE);
-  }
-
-  bool follow = false;
-  int input;
-
-  //Using GetOpt to read in the values
-  opterr = 0;
-  while ((input = getopt(argc, argv, "hf")) != -1) {
-    switch (input)
-    {
-    case 'h':
-      usage();
-      exit(EXIT_SUCCESS);
-    case 'f':
-      follow = true;
-      break;
-    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);
-  }
-
-  //We need to get the size of the file using stat
-  hdfs::StatInfo stat_info;
-  hdfs::Status status = fs->GetFileInfo(uri.get_path(), stat_info);
-  if (!status.ok()) {
-    std::cerr << "Error: " << status.ToString() << std::endl;
-    exit(EXIT_FAILURE);
-  }
-
-  //Determine where to start reading
-  off_t offset = 0;
-  if(stat_info.length > TAIL_SIZE){
-    offset = stat_info.length - TAIL_SIZE;
-  }
-
-  do {
-    off_t current_length = (off_t) stat_info.length;
-    readFile(fs, uri.get_path(), offset, stdout, false);
-
-    //Exit if -f flag was not set
-    if(!follow){
-      break;
-    }
-
-    do{
-      //Sleep for the REFRESH_RATE
-      sleep(REFRESH_RATE);
-      //Use stat to check the new filesize.
-      status = fs->GetFileInfo(uri.get_path(), stat_info);
-      if (!status.ok()) {
-        std::cerr << "Error: " << status.ToString() << std::endl;
-        exit(EXIT_FAILURE);
-      }
-      //If file became longer, loop back and print the difference
-    }
-    while((off_t) stat_info.length <= current_length);
-  } while (true);
-
-  // Clean up static data and prevent valgrind memory leaks
-  google::protobuf::ShutdownProtobufLibrary();
-  return 0;
-}