You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ab...@apache.org on 2020/03/23 22:01:58 UTC

[kudu] branch master updated: KUDU-3079 Add MiniPostgres

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

abukor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 22c5402  KUDU-3079 Add MiniPostgres
22c5402 is described below

commit 22c5402b0f0b9957e307a7ad032ab118b2399c49
Author: Attila Bukor <ab...@apache.org>
AuthorDate: Sun Mar 22 23:34:31 2020 +0100

    KUDU-3079 Add MiniPostgres
    
    Apache Ranger needs a database to store privileges and supported
    databases currently are MySQL, PostgreSQL, Oracle and Amazon RDS[1].
    
    I ruled out Amazon RDS on the basis that it's cloud-based so we couldn't
    depend on it in integration tests in a self-contained way.
    
    This is only a build-time dependency (test-time really) and only for an
    optional feature, so licensing shouldn't be too much of a concern for
    any of them, but PostgreSQL License can even be included in ASF
    software[2]. It's also easy to build and configure and it doesn't
    otherwise matter which database is used under Ranger, so I went with
    Postgres.
    
    This patch adds the latest (at the time of the start of this work)
    PostgreSQL and PostgreSQL JDBC driver JARs to thirdparty and adds a
    MiniPostgres interface that tests can use to start Postgres, add users
    and create databases.
    
    The patch also adds two utility methods that MiniPostgres uses:
    
    - Subprocess::WaitAndCheckExitCode() is the same as Wait(), but it only
      returns OK if the exit code is 0. If the subprocess was killed, stopped
      or exited with a non-zero exit code, it fails.
    - GetRandomPort() in net_util.h that returns a random unused port.
    
    I also removed the BaseName(argv[0]) call in Subprocess as that doesn't
    seem to be the correct behavior according to exec(3) man page as Adar
    pointed out.
    
    [1] https://docs.cloudera.com/HDPDocuments/HDP3/HDP-3.1.5/installing-ranger/content/configuring_a_database_instance_for_ranger.html
    (couldn't find a list of supported databases on ranger.apache.org or on
    cwiki.apache.org)
    [2] https://www.apache.org/legal/resolved.html#category-a
    
    Change-Id: Iba40126aa60e0ecbc5ae10cc1328493194c345bc
    Reviewed-on: http://gerrit.cloudera.org:8080/15374
    Reviewed-by: Hao Hao <ha...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Attila Bukor <ab...@apache.org>
---
 CMakeLists.txt                               |   1 +
 build-support/dist_test.py                   |   6 ++
 build-support/run_dist_test.py               |  12 ++-
 src/kudu/{ranger => postgres}/CMakeLists.txt |  54 +++++------
 src/kudu/postgres/mini_postgres-test.cc      |  60 ++++++++++++
 src/kudu/postgres/mini_postgres.cc           | 136 +++++++++++++++++++++++++++
 src/kudu/postgres/mini_postgres.h            | 102 ++++++++++++++++++++
 src/kudu/ranger/CMakeLists.txt               |   3 +-
 src/kudu/tools/kudu-tool-test.cc             |   2 +-
 src/kudu/util/net/net_util-test.cc           |   6 ++
 src/kudu/util/net/net_util.cc                |  13 +++
 src/kudu/util/net/net_util.h                 |   7 ++
 src/kudu/util/subprocess.cc                  |  17 +++-
 src/kudu/util/subprocess.h                   |   4 +
 thirdparty/LICENSE.txt                       |   5 +
 thirdparty/build-definitions.sh              |  18 ++++
 thirdparty/build-thirdparty.sh               |  11 +++
 thirdparty/download-thirdparty.sh            |  15 +++
 thirdparty/vars.sh                           |   8 ++
 19 files changed, 442 insertions(+), 38 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index df592b1..d2bb441 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1434,6 +1434,7 @@ add_subdirectory(src/kudu/integration-tests)
 add_subdirectory(src/kudu/kserver)
 add_subdirectory(src/kudu/master)
 add_subdirectory(src/kudu/mini-cluster)
+add_subdirectory(src/kudu/postgres)
 add_subdirectory(src/kudu/ranger)
 add_subdirectory(src/kudu/rebalance)
 add_subdirectory(src/kudu/rpc)
diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index c32022e..5056e83 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -113,6 +113,12 @@ DEPS_FOR_ALL = \
      # Add the Kudu echo subprocess.
      "build/latest/bin/kudu-subprocess.jar",
 
+     # Add Postgres. These are symlinks to directories in thirdparty.
+     "build/latest/bin/postgres",
+     "build/latest/bin/postgres-lib",
+     "build/latest/bin/postgres-share",
+     "build/latest/bin/postgresql.jar",
+
      # Add the Kudu HMS plugin.
      "build/latest/bin/hms-plugin.jar",
 
diff --git a/build-support/run_dist_test.py b/build-support/run_dist_test.py
index f9b12c7..09404b9 100755
--- a/build-support/run_dist_test.py
+++ b/build-support/run_dist_test.py
@@ -153,13 +153,21 @@ def main():
   env['SENTRY_HOME'] = glob.glob(os.path.join(ROOT, "thirdparty/src/sentry-*"))[0]
   env['JAVA_HOME'] = glob.glob("/usr/lib/jvm/java-1.8.0-*")[0]
 
-  # Restore the symlinks to the chrony binaries; tests expect to find them in
-  # same directory as the test binaries themselves.
+  # Restore the symlinks to the chrony binaries and Postgres directories; tests
+  # expect to find them in same directory as the test binaries themselves.
   for bin_path in glob.glob(os.path.join(ROOT, "build/*/bin")):
     os.symlink(os.path.join(ROOT, "thirdparty/installed/common/bin/chronyc"),
                os.path.join(bin_path, "chronyc"))
     os.symlink(os.path.join(ROOT, "thirdparty/installed/common/sbin/chronyd"),
                os.path.join(bin_path, "chronyd"))
+    os.symlink(os.path.join(ROOT, "thirdparty/installed/common/bin"),
+               os.path.join(bin_path, "postgres"))
+    os.symlink(os.path.join(ROOT, "thirdparty/installed/common/lib"),
+               os.path.join(bin_path, "postgres-lib"))
+    os.symlink(os.path.join(ROOT, "thirdparty/installed/common/share/postgresql"),
+               os.path.join(bin_path, "postgres-share"))
+    os.symlink(os.path.join(ROOT, "thirdparty/installed/common/opt/jdbc/postgresql.jar"),
+               os.path.join(bin_path, "postgresql.jar"))
 
   env['LD_LIBRARY_PATH'] = ":".join(
     [os.path.join(ROOT, "build/dist-test-system-libs/")] +
diff --git a/src/kudu/ranger/CMakeLists.txt b/src/kudu/postgres/CMakeLists.txt
similarity index 54%
copy from src/kudu/ranger/CMakeLists.txt
copy to src/kudu/postgres/CMakeLists.txt
index f486a19..e9c97a5 100644
--- a/src/kudu/ranger/CMakeLists.txt
+++ b/src/kudu/postgres/CMakeLists.txt
@@ -16,42 +16,34 @@
 # under the License.
 
 #######################################
-# ranger_proto
+# mini_postgres
 #######################################
 
-PROTOBUF_GENERATE_CPP(
-  RANGER_PROTO_SRCS RANGER_PROTO_HDRS RANGER_PROTO_TGTS
-  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
-  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
-  PROTO_FILES ranger.proto)
-
-add_library(ranger_proto
-  ${RANGER_PROTO_SRCS}
-  ${RANGER_PROTO_HDRS})
-target_link_libraries(ranger_proto
-  protobuf
+execute_process(COMMAND ln -nsf
+  "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/bin"
+  "${EXECUTABLE_OUTPUT_PATH}/postgres")
+execute_process(COMMAND ln -nsf
+  "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/lib"
+  "${EXECUTABLE_OUTPUT_PATH}/postgres-lib")
+execute_process(COMMAND ln -nsf
+  "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/share/postgresql"
+  "${EXECUTABLE_OUTPUT_PATH}/postgres-share")
+execute_process(COMMAND ln -nsf
+  "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/opt/jdbc/postgresql.jar"
+  "${EXECUTABLE_OUTPUT_PATH}/postgresql.jar")
+
+add_library(mini_postgres
+  mini_postgres.cc)
+target_link_libraries(mini_postgres
+  gutil
+  kudu_test_util
+  kudu_util
 )
 
-##############################
-# kudu_ranger
-##############################
-
-set(RANGER_SRCS
-  ranger_client.cc)
-set(RANGER_DEPS
-  gflags
-  kudu_subprocess
-  ranger_proto
-  security)
-
-add_library(kudu_ranger ${RANGER_SRCS})
-target_link_libraries(kudu_ranger ${RANGER_DEPS})
-
 #######################################
-# Unit tests
+# unit tests
 #######################################
 
-SET_KUDU_TEST_LINK_LIBS(
-  kudu_ranger)
+SET_KUDU_TEST_LINK_LIBS(mini_postgres)
 
-ADD_KUDU_TEST(ranger_client-test)
+ADD_KUDU_TEST(mini_postgres-test)
diff --git a/src/kudu/postgres/mini_postgres-test.cc b/src/kudu/postgres/mini_postgres-test.cc
new file mode 100644
index 0000000..c9efa72
--- /dev/null
+++ b/src/kudu/postgres/mini_postgres-test.cc
@@ -0,0 +1,60 @@
+// 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 "kudu/postgres/mini_postgres.h"
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+namespace postgres {
+class PostgresTest : public KuduTest {
+ public:
+  void SetUp() override {
+    postgres_.Start();
+  }
+
+ protected:
+  MiniPostgres postgres_;
+};
+
+TEST_F(PostgresTest, TestAddUser) {
+  ASSERT_OK((postgres_.AddUser("testuser", false)));
+}
+
+TEST_F(PostgresTest, TestCreateDatabaseNonExistentUser) {
+  ASSERT_TRUE(postgres_.CreateDb("testdb", "nonexistentuser").IsRuntimeError());
+}
+
+TEST_F(PostgresTest, TestCreateDatabaseExistingUserWithPersistence) {
+  ASSERT_OK(postgres_.AddUser("testuser", false));
+  ASSERT_OK(postgres_.CreateDb("testdb1", "testuser"));
+  // Restart Postgres to test persistence. The db and the user still has to
+  // exist after the restart, so creating a new database with 'testuser' needs
+  // to succeed but creating a new database with the same name must fail.
+  // Without persistence it would be the other way around
+  ASSERT_OK(postgres_.Stop());
+  ASSERT_OK(postgres_.Start());
+  ASSERT_OK(postgres_.CreateDb("testdb2", "testuser"));
+  ASSERT_TRUE(postgres_.CreateDb("testdb1", "testuser").IsRuntimeError());
+}
+
+} // namespace postgres
+} // namespace kudu
diff --git a/src/kudu/postgres/mini_postgres.cc b/src/kudu/postgres/mini_postgres.cc
new file mode 100644
index 0000000..9dcd898
--- /dev/null
+++ b/src/kudu/postgres/mini_postgres.cc
@@ -0,0 +1,136 @@
+// 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 "kudu/postgres/mini_postgres.h"
+
+#include <csignal>
+#include <ostream>
+#include <string>
+
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_util.h"
+
+using std::ifstream;
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+static constexpr int kPgStartTimeoutMs = 60000;
+
+namespace kudu {
+namespace postgres {
+
+MiniPostgres::~MiniPostgres() {
+  if (process_ && process_->IsStarted()) {
+    WARN_NOT_OK(Stop(),"Unable to stop postgres");
+  }
+}
+
+Status MiniPostgres::Start() {
+  Env* env = Env::Default();
+
+  VLOG(1) << "Starting Postgres";
+  string pgr = pg_root();
+  if (!env->FileExists(pgr)) {
+    // This is our first time running. Set up our directories, config files,
+    // and port.
+    LOG(INFO) << "Running initdb...";
+    Subprocess initdb({
+      JoinPathSegments(bin_dir_, "postgres/initdb"),
+      "-D", pgr, "-L", JoinPathSegments(bin_dir_, "postgres-share")
+    });
+    RETURN_NOT_OK_PREPEND(initdb.Start(), "failed to start initdb");
+    RETURN_NOT_OK_PREPEND(initdb.Wait(), "failed to wait on initdb");
+
+    // Postgres doesn't support binding to 0 so we need to get a random unused
+    // port and persist that to the config file.
+    RETURN_NOT_OK(GetRandomPort(&port_));
+    RETURN_NOT_OK(CreateConfigs());
+  }
+
+  process_.reset(new Subprocess({
+      JoinPathSegments(bin_dir_, "postgres/postgres"),
+      "-D", pgr}));
+  // LIBDIR needs to point to the directory containing the Postgres libraries,
+  // otherwise it defaults to /usr/lib/postgres.
+  process_->SetEnvVars({
+      { "LIBDIR", JoinPathSegments(bin_dir_, "postgres-lib") }
+  });
+  RETURN_NOT_OK(process_->Start());
+
+  const string ip = "127.0.0.1";
+  Status wait = WaitForTcpBind(process_->pid(), &port_, ip,
+                               MonoDelta::FromMilliseconds(kPgStartTimeoutMs));
+  if (!wait.ok()) {
+    // TODO(abukor): implement retry with a different port if it can't bind
+    WARN_NOT_OK(process_->Kill(SIGINT), "failed to send SIGINT to Postgres");
+  } else {
+    LOG(INFO) << "Postgres bound to " << port_;
+  }
+  return wait;
+}
+
+Status MiniPostgres::Stop() {
+  return process_->KillAndWait(SIGTERM);
+}
+
+Status MiniPostgres::AddUser(const string& user, bool super) {
+  Subprocess add({
+    JoinPathSegments(bin_dir_, "postgres/createuser"),
+    user,
+    Substitute("--$0superuser", super ? "" : "no-"),
+    "-p", SimpleItoa(port_),
+  });
+  RETURN_NOT_OK(add.Start());
+  return add.WaitAndCheckExitCode();
+}
+
+Status MiniPostgres::CreateDb(const string& db, const string& owner) {
+  Subprocess createdb({
+    JoinPathSegments(bin_dir_, "postgres/createdb"),
+    "-O", owner, db,
+    "-p", SimpleItoa(port_),
+  });
+  RETURN_NOT_OK(createdb.Start());
+  return createdb.WaitAndCheckExitCode();
+}
+
+Status MiniPostgres::CreateConfigs() {
+  Env* env = Env::Default();
+  // <pg_root>/postgresql.conf is generated by initdb in a previous step. We
+  // append the port to it.
+  string config_file = JoinPathSegments(pg_root(), "postgresql.conf");
+  faststring config;
+  ReadFileToString(env, config_file, &config);
+  config.append(Substitute("\nport=$0\n", port_));
+  unique_ptr<WritableFile> file;
+  RETURN_NOT_OK(env->NewWritableFile(config_file, &file));
+  RETURN_NOT_OK(file->Append(config));
+  return file->Close();
+}
+
+} // namespace postgres
+} // namespace kudu
diff --git a/src/kudu/postgres/mini_postgres.h b/src/kudu/postgres/mini_postgres.h
new file mode 100644
index 0000000..da8a0fd
--- /dev/null
+++ b/src/kudu/postgres/mini_postgres.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.
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/subprocess.h" // IWYU pragma: keep
+
+namespace kudu {
+namespace postgres {
+
+// Wrapper around Postgres to be used in MiniCluster for services requiring a
+// database connection (e.g. Apache Ranger).
+class MiniPostgres {
+ public:
+  MiniPostgres()
+    : MiniPostgres(GetTestDataDirectory()) {}
+
+  ~MiniPostgres();
+
+  explicit MiniPostgres(std::string data_root)
+    : data_root_(std::move(data_root)),
+      bin_dir_(GetBinDir()) {}
+
+  Status Start();
+  Status Stop();
+
+  // Creates a Postgres user with the specified name. If super is true, the
+  // created user will be a superuser, meaning all permission checks will be
+  // bypasssed[1].
+  //
+  // 1. https://www.postgresql.org/docs/12/role-attributes.html
+  Status AddUser(const std::string& user, bool super);
+
+  // Creates a database with the specified name. The owner has privileges to
+  // remove the database with all objects in it, even if they have different
+  // owners[1].
+  //
+  // 1. https://www.postgresql.org/docs/12/manage-ag-createdb.html
+  Status CreateDb(const std::string& db, const std::string& owner);
+
+  uint16_t bound_port() const {
+    CHECK_NE(0, port_);
+    return port_;
+  }
+
+  std::string pg_root() const {
+    return JoinPathSegments(data_root_, "postgres");
+  }
+
+  std::string pg_bin_dir() const {
+    return JoinPathSegments(bin_dir_, "postgres");
+  }
+
+ private:
+  static std::string GetBinDir() {
+    Env* env = Env::Default();
+    std::string exe;
+    CHECK_OK(env->GetExecutablePath(&exe));
+    return DirName(exe);
+  }
+
+  // 'pg_root' is the subdirectory in which the Postgres data files will live.
+  Status CreateConfigs();
+
+  // Directory in which to put all our stuff.
+  const std::string data_root_;
+
+  // Directory that has the Postgres binary.
+  // This may be in the thirdparty build, or may be shared across tests. As
+  // such, its contents should be treated as read-only.
+  const std::string bin_dir_;
+
+  std::unique_ptr<kudu::Subprocess> process_;
+  uint16_t port_ = 0;
+};
+
+} // namespace postgres
+} // namespace kudu
diff --git a/src/kudu/ranger/CMakeLists.txt b/src/kudu/ranger/CMakeLists.txt
index f486a19..2ad305e 100644
--- a/src/kudu/ranger/CMakeLists.txt
+++ b/src/kudu/ranger/CMakeLists.txt
@@ -52,6 +52,7 @@ target_link_libraries(kudu_ranger ${RANGER_DEPS})
 #######################################
 
 SET_KUDU_TEST_LINK_LIBS(
-  kudu_ranger)
+  kudu_ranger
+  mini_postgres)
 
 ADD_KUDU_TEST(ranger_client-test)
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index c668ea8..a3ec3c6 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -355,7 +355,7 @@ class ToolTest : public KuduTest {
     RunTool(arg_str, nullptr, nullptr, nullptr, /* stderr_lines = */ &err_lines);
     ASSERT_GE(err_lines.size(), 3) << err_lines;
     ASSERT_EQ(expected_status.ToString(), err_lines[0]);
-    ASSERT_STR_MATCHES(err_lines[2], "Usage: kudu.*");
+    ASSERT_STR_MATCHES(err_lines[2], "Usage: .*kudu.*");
   }
 
   void RunFsCheck(const string& arg_str,
diff --git a/src/kudu/util/net/net_util-test.cc b/src/kudu/util/net/net_util-test.cc
index 8ad0fba..26fbeec 100644
--- a/src/kudu/util/net/net_util-test.cc
+++ b/src/kudu/util/net/net_util-test.cc
@@ -207,4 +207,10 @@ TEST_F(NetUtilTest, TestGetFQDN) {
   LOG(INFO) << "fqdn is " << fqdn;
 }
 
+TEST_F(NetUtilTest, TestGetRandomPort) {
+  uint16_t port;
+  ASSERT_OK(GetRandomPort(&port));
+  LOG(INFO) << "Random port is " << port;
+}
+
 } // namespace kudu
diff --git a/src/kudu/util/net/net_util.cc b/src/kudu/util/net/net_util.cc
index 74653dd..6599f5c 100644
--- a/src/kudu/util/net/net_util.cc
+++ b/src/kudu/util/net/net_util.cc
@@ -51,6 +51,7 @@
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/subprocess.h"
@@ -436,6 +437,18 @@ Status HostPortFromSockaddrReplaceWildcard(const Sockaddr& addr, HostPort* hp) {
   return Status::OK();
 }
 
+Status GetRandomPort(uint16_t* port) {
+  Sockaddr address;
+  address.ParseString("127.0.0.1", 0);
+  Socket listener;
+  RETURN_NOT_OK(listener.Init(0));
+  RETURN_NOT_OK(listener.Bind(address));
+  Sockaddr listen_address;
+  RETURN_NOT_OK(listener.GetSocketAddress(&listen_address));
+  *port = listen_address.port();
+  return Status::OK();
+}
+
 void TryRunLsof(const Sockaddr& addr, vector<string>* log) {
 #if defined(__APPLE__)
   string cmd = strings::Substitute(
diff --git a/src/kudu/util/net/net_util.h b/src/kudu/util/net/net_util.h
index b904b00..3a561c0 100644
--- a/src/kudu/util/net/net_util.h
+++ b/src/kudu/util/net/net_util.h
@@ -232,6 +232,13 @@ enum class BindMode {
   LOOPBACK
 };
 
+// Gets a random port from the ephemeral range by binding to port 0 and letting
+// the kernel choose an unused one from the ephemeral port range. The socket is
+// then immediately closed and it remains in TIME_WAIT for 2*tcp_fin_timeout (by
+// default 2*60=120 seconds). The kernel won't assign this port until it's in
+// TIME_WAIT but it can still be used by binding it explicitly.
+Status GetRandomPort(uint16_t* port);
+
 #if defined(__APPLE__)
   static constexpr const BindMode kDefaultBindMode = BindMode::LOOPBACK;
 #else
diff --git a/src/kudu/util/subprocess.cc b/src/kudu/util/subprocess.cc
index 68f1560..c629e60 100644
--- a/src/kudu/util/subprocess.cc
+++ b/src/kudu/util/subprocess.cc
@@ -52,7 +52,6 @@
 #include "kudu/util/errno.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/monotime.h"
-#include "kudu/util/path_util.h"
 #include "kudu/util/signal.h"
 #include "kudu/util/status.h"
 #include "kudu/util/stopwatch.h"
@@ -266,8 +265,6 @@ Subprocess::Subprocess(vector<string> argv, int sig_on_destruct)
       fd_state_(),
       child_fds_(),
       sig_on_destruct_(sig_on_destruct) {
-  // By convention, the first argument in argv is the base name of the program.
-  argv_[0] = BaseName(argv_[0]);
 
   fd_state_[STDIN_FILENO]   = PIPED;
   fd_state_[STDOUT_FILENO]  = SHARED;
@@ -553,6 +550,20 @@ Status Subprocess::WaitNoBlock(int* wait_status) {
   return DoWait(wait_status, NON_BLOCKING);
 }
 
+Status Subprocess::WaitAndCheckExitCode() {
+  int wait_status;
+  RETURN_NOT_OK(DoWait(&wait_status, BLOCKING));
+  int exit_status;
+  string info_str;
+
+  RETURN_NOT_OK(GetExitStatus(&exit_status, &info_str));
+
+  return exit_status == 0
+    ? Status::OK()
+    : Status::RuntimeError(Substitute("Exit code: $0 ($1)",
+                                      exit_status, info_str));
+}
+
 Status Subprocess::GetProcfsState(int pid, ProcfsState* state) {
   faststring data;
   string filename = Substitute("/proc/$0/stat", pid);
diff --git a/src/kudu/util/subprocess.h b/src/kudu/util/subprocess.h
index 25947a9..d0561ef 100644
--- a/src/kudu/util/subprocess.h
+++ b/src/kudu/util/subprocess.h
@@ -112,6 +112,10 @@ class Subprocess {
   // exit code.
   Status WaitNoBlock(int* wait_status = nullptr) WARN_UNUSED_RESULT;
 
+  // Like Wait, but it also checks the exit code is 0. If it's not, or if it's
+  // not a clean exit, it returns RemoteError.
+  Status WaitAndCheckExitCode() WARN_UNUSED_RESULT;
+
   // Send a signal to the subprocess.
   // Note that this does not reap the process -- you must still Wait()
   // in order to reap it. Only call after starting.
diff --git a/thirdparty/LICENSE.txt b/thirdparty/LICENSE.txt
index a1845b8..d82455d 100644
--- a/thirdparty/LICENSE.txt
+++ b/thirdparty/LICENSE.txt
@@ -661,3 +661,8 @@ NOTE: build-time dependency
 thirdparty/src/gumbo-query-*/: MIT license
 Source: https://github.com/lazytiger/gumbo-query
 NOTE: build-time dependency
+
+--------------------------------------------------------------------------------
+thirdparty/src/postgresql-*/: PostgreSQL license
+Source: https://postgresql.org
+NOTE: build-time dependency
diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index 56e46dc..0f88333 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -1025,3 +1025,21 @@ build_gumbo_query() {
   ${NINJA:-make} -j$PARALLEL $EXTRA_MAKEFLAGS install
   popd
 }
+
+build_postgres() {
+  POSTGRES_BDIR=$TP_BUILD_DIR/$POSTGRES_NAME$MODE_SUFFIX
+  mkdir -p $POSTGRES_BDIR
+  pushd $POSTGRES_BDIR
+
+  # We don't need extra features like readline and zlib so so let's just
+  # simplify build.
+  CFLAGS="$EXTRA_CFLAGS" \
+    LDFLAGS="$EXTRA_LDFLAGS" \
+    $POSTGRES_SOURCE/configure \
+    --prefix=$PREFIX \
+    --without-readline \
+    --without-zlib
+
+  make -j$PARALLEL $EXTRA_MAKEFLAGS install
+  popd
+}
diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index f37ac54..ad8b7bc 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -103,6 +103,8 @@ else
       "chrony")       F_CHRONY=1 ;;
       "gumbo-parser") F_GUMBO_PARSER=1 ;;
       "gumbo-query")  F_GUMBO_QUERY=1 ;;
+      "postgres")     F_POSTGRES=1 ;;
+      "psql-jdbc")    F_POSTGRES_JDBC=1 ;;
       *)              echo "Unknown module: $arg"; exit 1 ;;
     esac
   done
@@ -252,6 +254,15 @@ if [ -n "$F_COMMON" -o -n "$F_CHRONY" ]; then
   build_chrony
 fi
 
+if [ -n "$F_COMMON" -o -n "$F_POSTGRES" ]; then
+  build_postgres
+fi
+
+if [ -n "$F_COMMON" -o -n "$F_POSTGRES_JDBC" ]; then
+  mkdir -p $PREFIX/opt/jdbc
+  ln -nsf $POSTGRES_JDBC_SOURCE/$POSTGRES_JDBC_NAME.jar $PREFIX/opt/jdbc/postgresql.jar
+fi
+
 # 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
diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh
index 88cda30..1a51aae 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -104,6 +104,9 @@ fetch_and_expand() {
         rm "$FILENAME"
         continue
       fi
+    elif [[ "$FILENAME" =~ \.jar$ ]]; then
+      mkdir ${FILENAME%.jar}
+      cp $FILENAME ${FILENAME%.jar}/
     else
       echo "Error: unknown file format: $FILENAME"
       exit 1
@@ -444,5 +447,17 @@ fetch_and_patch \
  $GUMBO_QUERY_PATCHLEVEL \
  "patch -p1 < $TP_DIR/patches/gumbo-query-namespace.patch"
 
+POSTGRES_PATCHLEVEL=0
+fetch_and_patch \
+ $POSTGRES_NAME.tar.gz \
+ $POSTGRES_SOURCE \
+ $POSTGRES_PATCHLEVEL
+
+POSTGRES_JDBC_PATCHLEVEL=0
+fetch_and_patch \
+ $POSTGRES_JDBC_NAME.jar \
+ $POSTGRES_JDBC_SOURCE \
+ $POSTGRES_JDBC_PATCHLEVEL
+
 echo "---------------"
 echo "Thirdparty dependencies downloaded successfully"
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index b182438..ac58bb2 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -254,3 +254,11 @@ GUMBO_PARSER_SOURCE=$TP_SOURCE_DIR/$GUMBO_PARSER_NAME
 GUMBO_QUERY_VERSION=c9f10880b645afccf4fbcd11d2f62a7c01222d2e
 GUMBO_QUERY_NAME=gumbo-query-$GUMBO_QUERY_VERSION
 GUMBO_QUERY_SOURCE=$TP_SOURCE_DIR/$GUMBO_QUERY_NAME
+
+POSTGRES_VERSION=12.2
+POSTGRES_NAME=postgresql-$POSTGRES_VERSION
+POSTGRES_SOURCE=$TP_SOURCE_DIR/$POSTGRES_NAME
+
+POSTGRES_JDBC_VERSION=42.2.10
+POSTGRES_JDBC_NAME=postgresql-$POSTGRES_JDBC_VERSION
+POSTGRES_JDBC_SOURCE=$TP_SOURCE_DIR/$POSTGRES_JDBC_NAME