You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2018/09/27 00:35:02 UTC

[2/4] kudu git commit: KUDU-428: add Sentry to thirdparty, mini-sentry

KUDU-428: add Sentry to thirdparty, mini-sentry

This commit adds Sentry to thirdparty, and fills out the MiniSentry
class with an initial implementation. Notable features that aren't
implemented:

- Stripped Sentry packaging. I've put an unmodified version of Sentry
  2.0.1 into thirdparty. It weighs in at almost 200MiB and takes about
  5s to startup on my laptop. We will probably want to add a stripped
  version later to reduce both of these.

- Kerberos support for mini-sentry. Right now Kerberos is disabled,
  which is an atypical configuration. A follow-up commit will add a
  Kerberos support configuration taking advantage of the mini KDC.

- The mini Sentry is not yet configured with the location of the HMS,
  which will be necessary to do anything non-trivial with it.

Change-Id: I03f39cf9b2c813c0c305d085e1ad3851636326f5
Reviewed-on: http://gerrit.cloudera.org:8080/11347
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Hao Hao <ha...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: fad56a6f85a4707f075f2334de21fb9025011795
Parents: bfdcd97
Author: Dan Burkert <da...@apache.org>
Authored: Thu Aug 23 16:40:24 2018 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Thu Sep 27 00:34:17 2018 +0000

----------------------------------------------------------------------
 build-support/dist_test.py            |   9 +-
 build-support/run_dist_test.py        |   3 +-
 src/kudu/hms/mini_hms.cc              |  37 ++-----
 src/kudu/sentry/CMakeLists.txt        |  10 ++
 src/kudu/sentry/mini_sentry.cc        | 154 +++++++++++++++++++++++++++++
 src/kudu/sentry/mini_sentry.h         |  47 +++++++++
 src/kudu/sentry/sentry_client-test.cc |  19 ++--
 src/kudu/util/test_util.cc            |  16 +++
 src/kudu/util/test_util.h             |   9 ++
 thirdparty/build-thirdparty.sh        |  10 +-
 thirdparty/download-thirdparty.sh     |   6 ++
 thirdparty/vars.sh                    |   4 +
 12 files changed, 281 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/build-support/dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index c1176fe..b2e7db9 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -97,12 +97,13 @@ DEPS_FOR_ALL = \
      # Tests that require tooling require this.
      "build/latest/bin/kudu",
 
-     # The HMS tests require the Hadoop and Hive libraries. These files are just
-     # symlinks, but dist-test will copy the entire directories they point to.
-     # The symlinks themselves won't be recreated, so we point to them with
-     # environment variables in run_dist_test.py.
+     # The HMS and Sentry tests require the Hadoop, Hive, and Sentry libraries.
+     # These files are just symlinks, but dist-test will copy the entire
+     # directories they point to.  The symlinks themselves won't be recreated,
+     # so we point to them with environment variables in run_dist_test.py.
      "build/latest/bin/hive-home",
      "build/latest/bin/hadoop-home",
+     "build/latest/bin/sentry-home",
 
      # Add the Kudu HMS plugin.
      "build/latest/bin/hms-plugin.jar",

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/build-support/run_dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/run_dist_test.py b/build-support/run_dist_test.py
index 297075d..1edcce2 100755
--- a/build-support/run_dist_test.py
+++ b/build-support/run_dist_test.py
@@ -147,9 +147,10 @@ def main():
       env[var_name] = os.environ.get(var_name, "") + " external_symbolizer_path=" + symbolizer_path
 
   # Add environment variables for Java dependencies. These environment variables
-  # are used in mini_hms.cc.
+  # are used in mini_hms.cc and mini_sentry.cc.
   env['HIVE_HOME'] = glob.glob(os.path.join(ROOT, "thirdparty/src/hive-*"))[0]
   env['HADOOP_HOME'] = glob.glob(os.path.join(ROOT, "thirdparty/src/hadoop-*"))[0]
+  env['SENTRY_HOME'] = glob.glob(os.path.join(ROOT, "thirdparty/src/apache-sentry-*"))[0]
   env['JAVA_HOME'] = glob.glob("/usr/lib/jvm/java-1.8.0-*")[0]
 
   env['LD_LIBRARY_PATH'] = ":".join(

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/hms/mini_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/mini_hms.cc b/src/kudu/hms/mini_hms.cc
index cc7e6bc..253de09 100644
--- a/src/kudu/hms/mini_hms.cc
+++ b/src/kudu/hms/mini_hms.cc
@@ -19,7 +19,6 @@
 
 #include <algorithm>
 #include <csignal>
-#include <cstdlib>
 #include <map>
 #include <memory>
 #include <ostream>
@@ -34,7 +33,6 @@
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/stopwatch.h"
-#include "kudu/util/string_case.h"
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_util.h"
 
@@ -76,25 +74,9 @@ void MiniHms::EnableKerberos(string krb5_conf,
 }
 
 void MiniHms::SetDataRoot(string data_root) {
-  data_root_ = data_root;
+  data_root_ = std::move(data_root);
 }
 
-namespace {
-Status FindHomeDir(const char* name, const string& bin_dir, string* home_dir) {
-  string name_upper;
-  ToUpperCase(name, &name_upper);
-
-  string env_var = Substitute("$0_HOME", name_upper);
-  const char* env = std::getenv(env_var.c_str());
-  *home_dir = env == nullptr ? JoinPathSegments(bin_dir, Substitute("$0-home", name)) : env;
-
-  if (!Env::Default()->FileExists(*home_dir)) {
-    return Status::NotFound(Substitute("$0 directory does not exist", env_var), *home_dir);
-  }
-  return Status::OK();
-}
-} // anonymous namespace
-
 Status MiniHms::Start() {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, kHmsStartTimeoutMs / 2, "Starting HMS");
   CHECK(!hms_process_);
@@ -292,13 +274,13 @@ Status MiniHms::CreateHiveSite() const {
 </configuration>
   )";
 
-  string file_contents = strings::Substitute(kFileTemplate,
-                                             notification_log_ttl_.ToSeconds(),
-                                             data_root_,
-                                             !keytab_file_.empty(),
-                                             keytab_file_,
-                                             service_principal_,
-                                             SaslProtection::name_of(protection_));
+  string file_contents = Substitute(kFileTemplate,
+                                    notification_log_ttl_.ToSeconds(),
+                                    data_root_,
+                                    !keytab_file_.empty(),
+                                    keytab_file_,
+                                    service_principal_,
+                                    SaslProtection::name_of(protection_));
 
   return WriteStringToFile(Env::Default(),
                            file_contents,
@@ -323,8 +305,7 @@ Status MiniHms::CreateCoreSite() const {
 </configuration>
   )";
 
-  string file_contents = strings::Substitute(kFileTemplate,
-                                             keytab_file_.empty() ? "simple" : "kerberos");
+  string file_contents = Substitute(kFileTemplate, keytab_file_.empty() ? "simple" : "kerberos");
 
   return WriteStringToFile(Env::Default(),
                            file_contents,

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/sentry/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/CMakeLists.txt b/src/kudu/sentry/CMakeLists.txt
index 8c75614..8e96c97 100644
--- a/src/kudu/sentry/CMakeLists.txt
+++ b/src/kudu/sentry/CMakeLists.txt
@@ -45,6 +45,16 @@ target_link_libraries(kudu_sentry ${SENTRY_DEPS})
 # mini_sentry
 ##############################
 
+execute_process(COMMAND ln -nsf
+                "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/opt/sentry"
+                "${EXECUTABLE_OUTPUT_PATH}/sentry-home")
+execute_process(COMMAND ln -nsf
+                "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/opt/hadoop"
+                "${EXECUTABLE_OUTPUT_PATH}/hadoop-home")
+execute_process(COMMAND ln -nsf
+                "${JAVA_HOME}"
+                "${EXECUTABLE_OUTPUT_PATH}/java-home")
+
 set(MINI_SENTRY_SRCS
   mini_sentry.cc)
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/sentry/mini_sentry.cc
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/mini_sentry.cc b/src/kudu/sentry/mini_sentry.cc
index 1b00099..865fd2d 100644
--- a/src/kudu/sentry/mini_sentry.cc
+++ b/src/kudu/sentry/mini_sentry.cc
@@ -17,7 +17,161 @@
 
 #include "kudu/sentry/mini_sentry.h"
 
+#include <algorithm>
+#include <csignal>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_util.h"
+
+using std::map;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+static constexpr int kSentryStartTimeoutMs = 60000;
+
 namespace kudu {
 namespace sentry {
+
+MiniSentry::MiniSentry() {
+}
+
+MiniSentry::~MiniSentry() {
+  WARN_NOT_OK(Stop(), "Failed to stop MiniSentry");
+}
+
+Status MiniSentry::Start() {
+  SCOPED_LOG_SLOW_EXECUTION(WARNING, kSentryStartTimeoutMs / 2, "Starting Sentry");
+  CHECK(!sentry_process_);
+
+  VLOG(1) << "Starting Sentry";
+
+  Env* env = Env::Default();
+
+  string exe;
+  RETURN_NOT_OK(env->GetExecutablePath(&exe));
+  const string bin_dir = DirName(exe);
+
+  string hadoop_home;
+  string sentry_home;
+  string java_home;
+  RETURN_NOT_OK(FindHomeDir("hadoop", bin_dir, &hadoop_home));
+  RETURN_NOT_OK(FindHomeDir("sentry", bin_dir, &sentry_home));
+  RETURN_NOT_OK(FindHomeDir("java", bin_dir, &java_home));
+
+  auto tmp_dir = GetTestDataDirectory();
+
+  RETURN_NOT_OK(CreateSentrySite(tmp_dir));
+
+  map<string, string> env_vars {
+      { "JAVA_HOME", java_home },
+      { "HADOOP_HOME", hadoop_home },
+  };
+
+  // Start Sentry.
+  sentry_process_.reset(new Subprocess({
+      Substitute("$0/bin/sentry", sentry_home),
+      "--command", "service",
+      "--conffile", JoinPathSegments(tmp_dir, "sentry-site.xml"),
+  }));
+
+  sentry_process_->SetEnvVars(env_vars);
+  RETURN_NOT_OK(sentry_process_->Start());
+
+  // Wait for Sentry to start listening on its ports and commencing operation.
+  VLOG(1) << "Waiting for Sentry ports";
+  Status wait = WaitForTcpBind(sentry_process_->pid(), &port_,
+                               MonoDelta::FromMilliseconds(kSentryStartTimeoutMs));
+  if (!wait.ok()) {
+    WARN_NOT_OK(sentry_process_->Kill(SIGQUIT), "failed to send SIGQUIT to Sentry");
+  }
+  return wait;
+}
+
+Status MiniSentry::Stop() {
+  if (sentry_process_) {
+    VLOG(1) << "Stopping Sentry";
+    unique_ptr<Subprocess> proc = std::move(sentry_process_);
+    RETURN_NOT_OK_PREPEND(proc->KillAndWait(SIGTERM), "failed to stop the Sentry service");
+  }
+  return Status::OK();
+}
+
+Status MiniSentry::Pause() {
+  CHECK(sentry_process_);
+  VLOG(1) << "Pausing Sentry";
+  RETURN_NOT_OK_PREPEND(sentry_process_->Kill(SIGSTOP),
+                        "failed to pause the Sentry service");
+  return Status::OK();
+}
+
+Status MiniSentry::Resume() {
+  CHECK(sentry_process_);
+  VLOG(1) << "Resuming Sentry";
+  RETURN_NOT_OK_PREPEND(sentry_process_->Kill(SIGCONT),
+                        "failed to unpause the Sentry service");
+  return Status::OK();
+}
+
+Status MiniSentry::CreateSentrySite(const string& tmp_dir) const {
+
+  // - sentry.store.jdbc.url
+  // - sentry.store.jdbc.password
+  //     Configures Sentry to use a local in-process Derby instance with a dummy
+  //     password value.
+  //
+  // - datanucleus.schema.autoCreateAll
+  // - sentry.verify.schema.version
+  //     Allow Sentry to startup and run without first running the schemaTool.
+  static const string kFileTemplate = R"(
+<configuration>
+
+  <property>
+    <name>sentry.service.security.mode</name>
+    <value>none</value>
+  </property>
+
+  <property>
+    <name>sentry.store.jdbc.url</name>
+    <value>jdbc:derby:$0/sentry;create=true</value>
+  </property>
+
+  <property>
+    <name>sentry.store.jdbc.password</name>
+    <value>_</value>
+  </property>
+
+  <property>
+    <name>datanucleus.schema.autoCreateAll</name>
+    <value>true</value>
+  </property>
+
+  <property>
+   <name>sentry.verify.schema.version</name>
+    <value>false</value>
+  </property>
+</configuration>
+  )";
+
+  string file_contents = Substitute(kFileTemplate, tmp_dir);
+
+  return WriteStringToFile(Env::Default(),
+                           file_contents,
+                           JoinPathSegments(tmp_dir, "sentry-site.xml"));
+}
+
 } // namespace sentry
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/sentry/mini_sentry.h
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/mini_sentry.h b/src/kudu/sentry/mini_sentry.h
index adc081d..fe3ccca 100644
--- a/src/kudu/sentry/mini_sentry.h
+++ b/src/kudu/sentry/mini_sentry.h
@@ -17,11 +17,58 @@
 
 #pragma once
 
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/status.h"
+
 namespace kudu {
+
+class Subprocess;
+
 namespace sentry {
 
 class MiniSentry {
  public:
+
+  MiniSentry();
+
+  ~MiniSentry();
+
+  // Starts the mini Sentry service.
+  //
+  // If the MiniSentry has already been started and stopped, it will be restarted
+  // using the same listening port.
+  Status Start() WARN_UNUSED_RESULT;
+
+  // Stops the mini Sentry service.
+  Status Stop() WARN_UNUSED_RESULT;
+
+  // Pause the Sentry service.
+  Status Pause() WARN_UNUSED_RESULT;
+
+  // Unpause the Sentry service.
+  Status Resume() WARN_UNUSED_RESULT;
+
+  // Returns the address of the Sentry service. Should only be called after the
+  // Sentry service is started.
+  HostPort address() const {
+    return HostPort("127.0.0.1", port_);
+  }
+
+ private:
+
+  // Creates a sentry-site.xml for the mini Sentry.
+  Status CreateSentrySite(const std::string& tmp_dir) const WARN_UNUSED_RESULT;
+
+  // Waits for the metastore process to bind to a port.
+  Status WaitForSentryPorts() WARN_UNUSED_RESULT;
+
+  std::unique_ptr<Subprocess> sentry_process_;
+  uint16_t port_ = 0;
 };
 
 } // namespace sentry

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/sentry/sentry_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/sentry_client-test.cc b/src/kudu/sentry/sentry_client-test.cc
index 6293ac1..8f3c560 100644
--- a/src/kudu/sentry/sentry_client-test.cc
+++ b/src/kudu/sentry/sentry_client-test.cc
@@ -15,12 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/sentry/sentry_client.h"
-
-#include <utility>
-
 #include <gtest/gtest.h>
 
+#include "kudu/sentry/mini_sentry.h"
+#include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
 namespace kudu {
@@ -30,10 +28,15 @@ class SentryClientTest : public KuduTest {
  public:
 };
 
-TEST_F(SentryClientTest, ItWorks) {
-  SentryClient client;
-  std::move(client);
-}
+TEST_F(SentryClientTest, TestMiniSentryLifecycle) {
+  MiniSentry mini_sentry;
+  ASSERT_OK(mini_sentry.Start());
 
+  ASSERT_OK(mini_sentry.Stop());
+  ASSERT_OK(mini_sentry.Start());
+
+  ASSERT_OK(mini_sentry.Pause());
+  ASSERT_OK(mini_sentry.Resume());
+}
 } // namespace sentry
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index c960441..214964d 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -54,6 +54,7 @@
 #include "kudu/util/slice.h"
 #include "kudu/util/spinlock_profiling.h"
 #include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
 #include "kudu/util/subprocess.h"
 
 DEFINE_string(test_leave_files, "on_failure",
@@ -443,4 +444,19 @@ Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
   return WaitForBind(pid, port, "4UDP", timeout);
 }
 
+Status FindHomeDir(const string& name, const string& bin_dir, string* home_dir) {
+  string name_upper;
+  ToUpperCase(name, &name_upper);
+
+  string env_var = Substitute("$0_HOME", name_upper);
+  const char* env = std::getenv(env_var.c_str());
+  string dir = env == nullptr ? JoinPathSegments(bin_dir, Substitute("$0-home", name)) : env;
+
+  if (!Env::Default()->FileExists(dir)) {
+    return Status::NotFound(Substitute("$0 directory does not exist", env_var), dir);
+  }
+  *home_dir = dir;
+  return Status::OK();
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/src/kudu/util/test_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index 8090fbc..32dfacd 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -142,5 +142,14 @@ Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_
 // Waits for the subprocess to bind to any listening UDP port, and returns the port.
 Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
 
+// Find the home directory of a Java-style application, e.g. JAVA_HOME or
+// HADOOP_HOME.
+//
+// Checks the environment, or falls back to a symlink in the bin installation
+// directory.
+Status FindHomeDir(const std::string& name,
+                   const std::string& bin_dir,
+                   std::string* home_dir) WARN_UNUSED_RESULT;
+
 } // namespace kudu
 #endif

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/thirdparty/build-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index 8c2a44d..c13c95c 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -99,6 +99,7 @@ else
       "bison")        F_BISON=1 ;;
       "hadoop")       F_HADOOP=1 ;;
       "hive")         F_HIVE=1 ;;
+      "sentry")       F_SENTRY=1 ;;
       *)              echo "Unknown module: $arg"; exit 1 ;;
     esac
   done
@@ -244,8 +245,8 @@ if [ -n "$F_COMMON" -o -n "$F_BISON" ]; then
   build_bison
 fi
 
-# Install Hadoop and Hive by symlinking their source directories (which are
-# pre-built) into $PREFIX/opt.
+# Install Hadoop, Hive, and Sentry by symlinking their source directories (which
+# are pre-built) into $PREFIX/opt.
 if [ -n "$F_COMMON" -o -n "$F_HADOOP" ]; then
   mkdir -p $PREFIX/opt
   ln -nsf $HADOOP_SOURCE $PREFIX/opt/hadoop
@@ -256,6 +257,11 @@ if [ -n "$F_COMMON" -o -n "$F_HIVE" ]; then
   ln -nsf $HIVE_SOURCE $PREFIX/opt/hive
 fi
 
+if [ -n "$F_COMMON" -o -n "$F_SENTRY" ]; then
+  mkdir -p $PREFIX/opt
+  ln -nsf $SENTRY_SOURCE $PREFIX/opt/sentry
+fi
+
 ### Build C dependencies without instrumentation
 
 PREFIX=$PREFIX_DEPS

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/thirdparty/download-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh
index 7992065..9b19305 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -423,5 +423,11 @@ fetch_and_patch \
  $HADOOP_SOURCE \
  $HADOOP_PATCHLEVEL
 
+SENTRY_PATCHLEVEL=0
+fetch_and_patch \
+ $SENTRY_NAME.tar.gz \
+ $SENTRY_SOURCE \
+ $SENTRY_PATCHLEVEL
+
 echo "---------------"
 echo "Thirdparty dependencies downloaded successfully"

http://git-wip-us.apache.org/repos/asf/kudu/blob/fad56a6f/thirdparty/vars.sh
----------------------------------------------------------------------
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index 71beea6..fe25293 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -225,3 +225,7 @@ HIVE_SOURCE=$TP_SOURCE_DIR/$HIVE_NAME
 HADOOP_VERSION=2.8.2
 HADOOP_NAME=hadoop-$HADOOP_VERSION
 HADOOP_SOURCE=$TP_SOURCE_DIR/$HADOOP_NAME
+
+SENTRY_VERSION=2.0.1
+SENTRY_NAME=apache-sentry-$SENTRY_VERSION-bin
+SENTRY_SOURCE=$TP_SOURCE_DIR/$SENTRY_NAME