You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/20 12:46:41 UTC

[kudu] branch master updated (efc3f37 -> 02fa2d6)

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

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


    from efc3f37  KUDU-2871 (part 1): disable TLS 1.3.
     new 5dca59e  KUDU-2870: allow super-user to skip authz checks in Checksum
     new 1d1abbf  [tools] Add get/set extra-configs for CLI tools
     new 02fa2d6  [docs] Ubuntu 18.04 is a supported platform

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/installation.adoc                       |  4 +-
 src/kudu/integration-tests/security-itest.cc | 76 +++++++++++++++++++----
 src/kudu/server/server_base.cc               |  7 ++-
 src/kudu/server/server_base.h                |  3 +
 src/kudu/tools/kudu-admin-test.cc            | 90 ++++++++++++++++++++++++++++
 src/kudu/tools/kudu-tool-test.cc             | 32 ++++++++++
 src/kudu/tools/tool_action_table.cc          | 66 ++++++++++++++++++++
 src/kudu/tserver/tablet_service.cc           |  7 ++-
 8 files changed, 269 insertions(+), 16 deletions(-)


[kudu] 01/03: KUDU-2870: allow super-user to skip authz checks in Checksum

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5dca59eba4cb2717a2cf1dfbc5bd750fd2535366
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Wed Jun 19 14:05:59 2019 -0700

    KUDU-2870: allow super-user to skip authz checks in Checksum
    
    In order to allow for the Kudu CLI to run a checksum scan (a process
    which currently doesn't fetch an authz token from the Master), this
    patch allows Checksum to proceed if the requesting user is a super-user.
    
    Testing:
    - A test is added to run the CLI against a tserver that enforces
      fine-grained access control.
    - A new test is added to test the super-user permissions for the
      Checksum endpoint.
    
    Change-Id: I9da21f41702da747a081ab037d75865748d981a8
    Reviewed-on: http://gerrit.cloudera.org:8080/13681
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/integration-tests/security-itest.cc | 76 ++++++++++++++++++++++++----
 src/kudu/server/server_base.cc               |  7 ++-
 src/kudu/server/server_base.h                |  3 ++
 src/kudu/tools/kudu-tool-test.cc             | 30 +++++++++++
 src/kudu/tserver/tablet_service.cc           |  7 ++-
 5 files changed, 109 insertions(+), 14 deletions(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 92d7322..1003461 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -34,10 +34,11 @@
 #include "kudu/client/shared_ptr.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/common/schema.h"
+#include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/consensus.proxy.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
@@ -50,7 +51,9 @@
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tablet/key_value_test_schema.h"
+#include "kudu/tablet/tablet.pb.h"
 #include "kudu/tserver/tserver.pb.h"
+#include "kudu/tserver/tserver_service.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/env.h"
 #include "kudu/util/monotime.h"
@@ -82,6 +85,10 @@ using strings::Substitute;
 
 namespace kudu {
 
+static const char* kTableName = "test-table";
+static const Schema kTestSchema = CreateKeyValueTestSchema();
+static const KuduSchema kTestKuduSchema = client::KuduSchema::FromSchema(kTestSchema);
+
 class SecurityITest : public KuduTest {
  public:
   SecurityITest() {
@@ -111,6 +118,15 @@ class SecurityITest : public KuduTest {
     return proxy.SetFlag(req, &resp, &controller);
   }
 
+  Status CreateTestTable(const client::sp::shared_ptr<KuduClient>& client) {
+    unique_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
+    return table_creator->table_name(kTableName)
+        .set_range_partition_columns({ "key" })
+        .schema(&kTestKuduSchema)
+        .num_replicas(3)
+        .Create();
+  }
+
   // Create a table, insert a row, scan it back, and delete the table.
   void SmokeTestCluster();
 
@@ -129,7 +145,7 @@ class SecurityITest : public KuduTest {
     return proxy.TSHeartbeat(req, &resp, &rpc);
   }
 
-  Status TryListTablets() {
+  Status TryListTablets(vector<string>* tablet_ids = nullptr) {
     auto messenger = NewMessengerOrDie();
     const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
     tserver::TabletServerServiceProxy proxy(messenger, addr, addr.host());
@@ -137,7 +153,28 @@ class SecurityITest : public KuduTest {
     rpc::RpcController rpc;
     tserver::ListTabletsRequestPB req;
     tserver::ListTabletsResponsePB resp;
-    return proxy.ListTablets(req, &resp, &rpc);
+    RETURN_NOT_OK(proxy.ListTablets(req, &resp, &rpc));
+    if (tablet_ids) {
+      for (int i = 0; i < resp.status_and_schema_size(); i++) {
+        tablet_ids->emplace_back(resp.status_and_schema(i).tablet_status().tablet_id());
+      }
+    }
+    return Status::OK();
+  }
+
+  // Sends a request to checksum the given tablet without an authz token.
+  Status TryChecksumWithoutAuthzToken(const string& tablet_id) {
+    auto messenger = NewMessengerOrDie();
+    const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
+    tserver::TabletServerServiceProxy proxy(messenger, addr, addr.host());
+
+    rpc::RpcController rpc;
+    tserver::ChecksumRequestPB req;
+    tserver::NewScanRequestPB* scan = req.mutable_new_request();
+    scan->set_tablet_id(tablet_id);
+    RETURN_NOT_OK(SchemaToColumnPBs(kTestSchema, scan->mutable_projected_columns()));
+    tserver::ChecksumResponsePB resp;
+    return proxy.Checksum(req, &resp, &rpc);
   }
 
  private:
@@ -156,18 +193,11 @@ class SecurityITest : public KuduTest {
 };
 
 void SecurityITest::SmokeTestCluster() {
-  const char* kTableName = "test-table";
   client::sp::shared_ptr<KuduClient> client;
   ASSERT_OK(cluster_->CreateClient(nullptr, &client));
 
   // Create a table.
-  KuduSchema schema = client::KuduSchema::FromSchema(CreateKeyValueTestSchema());
-  gscoped_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
-  ASSERT_OK(table_creator->table_name(kTableName)
-            .set_range_partition_columns({ "key" })
-            .schema(&schema)
-            .num_replicas(3)
-            .Create());
+  ASSERT_OK(CreateTestTable(client));
 
   // Insert a row.
   client::sp::shared_ptr<KuduTable> table;
@@ -201,6 +231,30 @@ TEST_F(SecurityITest, TestAuthorizationOnListTablets) {
   ASSERT_OK(TryListTablets());
 }
 
+TEST_F(SecurityITest, TestAuthorizationOnChecksum) {
+  cluster_opts_.extra_tserver_flags.emplace_back("--tserver_enforce_access_control");
+  ASSERT_OK(StartCluster());
+  client::sp::shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+  ASSERT_OK(CreateTestTable(client));
+  vector<string> tablet_ids;
+  ASSERT_OK(TryListTablets(&tablet_ids));
+
+  // As a regular user, we shouldn't be authorized if we didn't send an authz
+  // token.
+  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
+  for (const auto& tablet_id : tablet_ids) {
+    Status s = TryChecksumWithoutAuthzToken(tablet_id);
+    ASSERT_STR_CONTAINS(s.ToString(), "Not authorized: no authorization token presented");
+  }
+  // As a super-user (e.g. if running the CLI as an admin), this should be
+  // allowed.
+  ASSERT_OK(cluster_->kdc()->Kinit("test-admin"));
+  for (const auto& tablet_id : tablet_ids) {
+    ASSERT_OK(TryChecksumWithoutAuthzToken(tablet_id));
+  }
+}
+
 // Test creating a table, writing some data, reading data, and dropping
 // the table.
 TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 6cd3075..636cde2 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -585,9 +585,12 @@ void ServerBase::LogUnauthorizedAccess(rpc::RpcContext* rpc) const {
                << " from " << rpc->requestor_string();
 }
 
+bool ServerBase::IsFromSuperUser(const rpc::RpcContext* rpc) {
+  return superuser_acl_.UserAllowed(rpc->remote_user().username());
+}
+
 bool ServerBase::Authorize(rpc::RpcContext* rpc, uint32_t allowed_roles) {
-  if ((allowed_roles & SUPER_USER) &&
-      superuser_acl_.UserAllowed(rpc->remote_user().username())) {
+  if ((allowed_roles & SUPER_USER) && IsFromSuperUser(rpc)) {
     return true;
   }
 
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index 587e446..6859e3d 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -119,6 +119,9 @@ class ServerBase {
     SERVICE_USER = 1 << 2
   };
 
+  // Returns whether or not the rpc is from a super-user.
+  bool IsFromSuperUser(const rpc::RpcContext* rpc);
+
   // Authorize an RPC. 'allowed_roles' is a bitset of which roles from the above
   // enum should be allowed to make hthe RPC.
   //
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 8761522..be454f6 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -4995,5 +4995,35 @@ TEST_P(Is343ReplicaUtilTest, Is343Cluster) {
   }
 }
 
+class AuthzTServerChecksumTest : public ToolTest {
+ public:
+  void SetUp() override {
+    ExternalMiniClusterOptions opts;
+    opts.extra_tserver_flags.emplace_back("--tserver_enforce_access_control=true");
+    NO_FATALS(StartExternalMiniCluster(std::move(opts)));
+  }
+};
+
+// Test the authorization of Checksum scans via the CLI.
+TEST_F(AuthzTServerChecksumTest, TestAuthorizeChecksum) {
+  // First, let's create a table.
+  const vector<string> loadgen_args = {
+    "perf", "loadgen",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    "--keep_auto_table",
+    "--num_rows_per_thread=0",
+  };
+  ASSERT_OK(RunKuduTool(loadgen_args));
+
+  // Running a checksum scan should succeed since the tool is run as the OS
+  // user, which is the default super-user.
+  const vector<string> checksum_args = {
+    "cluster", "ksck",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    "--checksum_scan"
+  };
+  ASSERT_OK(RunKuduTool(checksum_args));
+}
+
 } // namespace tools
 } // namespace kudu
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index b1b1792..c752dcd 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1933,7 +1933,12 @@ void TabletServiceImpl::Checksum(const ChecksumRequestPB* req,
   ScanResultChecksummer collector;
   bool has_more = false;
   TabletServerErrorPB::Code error_code = TabletServerErrorPB::UNKNOWN_ERROR;
-  if (FLAGS_tserver_enforce_access_control && req->has_new_request()) {
+  // TODO(KUDU-2870): the CLI tool doesn't currently fetch authz tokens when
+  // checksumming. Until it does, allow the super-user to avoid fine-grained
+  // privilege checking.
+  if (FLAGS_tserver_enforce_access_control &&
+      !server_->IsFromSuperUser(context) &&
+      req->has_new_request()) {
     const NewScanRequestPB& new_req = req->new_request();
     TokenPB token;
     if (!VerifyAuthzTokenOrRespond(server_->token_verifier(), req->new_request(),


[kudu] 02/03: [tools] Add get/set extra-configs for CLI tools

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1d1abbfb6a421df39843d6c3e4f18089b86e632e
Author: oclarms <oc...@gmail.com>
AuthorDate: Fri Jun 14 17:20:56 2019 +0800

    [tools] Add get/set extra-configs for CLI tools
    
    Change-Id: I90c790bbfe41a59f621157ff6b3f11d2b8f916e7
    Reviewed-on: http://gerrit.cloudera.org:8080/13649
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/tools/kudu-admin-test.cc   | 90 +++++++++++++++++++++++++++++++++++++
 src/kudu/tools/kudu-tool-test.cc    |  2 +
 src/kudu/tools/tool_action_table.cc | 66 +++++++++++++++++++++++++++
 3 files changed, 158 insertions(+)

diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 4be0f27..a9a0cc0 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -2251,5 +2251,95 @@ TEST_F(AdminCliTest, TestAuthzResetCacheNotImplemented) {
       "Not implemented: provider does not have privileges cache");
 }
 
+TEST_F(AdminCliTest, TestExtraConfig) {
+  NO_FATALS(BuildAndStart());
+
+  string master_address = cluster_->master()->bound_rpc_addr().ToString();
+
+  // Gets extra-configs when no extra config set.
+  {
+    string stdout, stderr;
+    Status s = RunKuduTool({
+      "table",
+      "get_extra_configs",
+      master_address,
+      kTableId
+    }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_EQ(stdout, " Configuration | Value\n"
+                      "---------------+-------\n");
+  }
+
+  // Sets "kudu.table.history_max_age_sec" to 3600.
+  {
+    ASSERT_TOOL_OK(
+      "table",
+      "set_extra_config",
+      master_address,
+      kTableId,
+      "kudu.table.history_max_age_sec",
+      "3600"
+    );
+  }
+
+  // Gets all extra-configs.
+  {
+    string stdout, stderr;
+    Status s = RunKuduTool({
+      "table",
+      "get_extra_configs",
+      master_address,
+      kTableId,
+    }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_STR_CONTAINS(stdout, "kudu.table.history_max_age_sec | 3600");
+  }
+
+  // Gets the specified extra-config, the configuration exists.
+  {
+    string stdout, stderr;
+    Status s = RunKuduTool({
+      "table",
+      "get_extra_configs",
+      master_address,
+      kTableId,
+      "-config_names=kudu.table.history_max_age_sec"
+    }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_STR_CONTAINS(stdout, "kudu.table.history_max_age_sec | 3600");
+  }
+
+  // Gets the duplicate extra-configs, the configuration exists.
+  {
+    string stdout, stderr;
+    Status s = RunKuduTool({
+      "table",
+      "get_extra_configs",
+      master_address,
+      kTableId,
+      "-config_names=kudu.table.history_max_age_sec,kudu.table.history_max_age_sec"
+      }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_EQ(stdout, "         Configuration          | Value\n"
+                      "--------------------------------+-------\n"
+                      " kudu.table.history_max_age_sec | 3600\n");
+  }
+
+  // Gets the specified extra-config, the configuration doesn't exists.
+  {
+    string stdout, stderr;
+    Status s = RunKuduTool({
+      "table",
+      "get_extra_configs",
+      master_address,
+      kTableId,
+      "-config_names=foobar"
+      }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_EQ(stdout, " Configuration | Value\n"
+                      "---------------+-------\n");
+  }
+}
+
 } // namespace tools
 } // namespace kudu
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index be454f6..5ac2c4a 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1107,6 +1107,8 @@ TEST_F(ToolTest, TestModeHelp) {
         "list.*List tables",
         "scan.*Scan rows from a table",
         "copy.*Copy table data to another table",
+        "set_extra_config.*Change a extra configuration value on a table",
+        "get_extra_configs.*Get the extra configuration properties for a table"
     };
     NO_FATALS(RunTestHelp("table", kTableModeRegexes));
   }
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 8502dc8..6ca7373 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -18,9 +18,12 @@
 #include <algorithm>
 #include <cstdint>
 #include <iostream>
+#include <map>
 #include <memory>
+#include <set>
 #include <string>
 #include <unordered_map>
+#include <utility>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
@@ -62,6 +65,7 @@ using kudu::client::internal::ReplicaController;
 using std::cerr;
 using std::cout;
 using std::endl;
+using std::set;
 using std::string;
 using std::unique_ptr;
 using std::vector;
@@ -81,6 +85,9 @@ DEFINE_bool(list_tablets, false,
 DEFINE_bool(modify_external_catalogs, true,
             "Whether to modify external catalogs, such as the Hive Metastore, "
             "when renaming or dropping a table.");
+DEFINE_string(config_names, "",
+              "Comma-separated list of configurations to display. "
+              "An empty value displays all configs.");
 DECLARE_bool(show_values);
 DECLARE_string(tables);
 
@@ -138,6 +145,8 @@ const char* const kNewTableNameArg = "new_table_name";
 const char* const kColumnNameArg = "column_name";
 const char* const kNewColumnNameArg = "new_column_name";
 const char* const kKeyArg = "primary_key";
+const char* const kConfigNameArg = "config_name";
+const char* const kConfigValueArg = "config_value";
 
 Status DeleteTable(const RunnerContext& context) {
   const string& table_name = FindOrDie(context.required_args, kTableNameArg);
@@ -418,6 +427,43 @@ Status CopyTable(const RunnerContext& context) {
   return scanner.StartCopy();
 }
 
+Status SetExtraConfig(const RunnerContext& context) {
+  const string& table_name = FindOrDie(context.required_args, kTableNameArg);
+  const string& config_name = FindOrDie(context.required_args, kConfigNameArg);
+  const string& config_value = FindOrDie(context.required_args, kConfigValueArg);
+
+  client::sp::shared_ptr<KuduClient> client;
+  RETURN_NOT_OK(CreateKuduClient(context, &client));
+  unique_ptr<KuduTableAlterer> alterer(client->NewTableAlterer(table_name));
+  alterer->AlterExtraConfig({ { config_name, config_value} });
+  return alterer->Alter();
+}
+
+Status GetExtraConfigs(const RunnerContext& context) {
+  const string& table_name = FindOrDie(context.required_args, kTableNameArg);
+  set<string> config_names = strings::Split(FLAGS_config_names, ",", strings::SkipEmpty());
+
+  client::sp::shared_ptr<KuduClient> client;
+  RETURN_NOT_OK(CreateKuduClient(context, &client));
+  client::sp::shared_ptr<KuduTable> table;
+  RETURN_NOT_OK(client->OpenTable(table_name, &table));
+
+  DataTable data_table({ "Configuration", "Value" });
+  if (config_names.empty()) {
+    for (const auto& extra_config : table->extra_configs()) {
+      data_table.AddRow({ extra_config.first, extra_config.second });
+    }
+  } else {
+    for (const auto& config_name : config_names) {
+      const string* config_value = FindOrNull(table->extra_configs(), config_name);
+      if (config_value) {
+        data_table.AddRow({ config_name, *config_value });
+      }
+    }
+  }
+  return data_table.PrintTo(cout);
+}
+
 } // anonymous namespace
 
 unique_ptr<Mode> BuildTableMode() {
@@ -513,6 +559,24 @@ unique_ptr<Mode> BuildTableMode() {
       .AddOptionalParameter("write_type")
       .Build();
 
+  unique_ptr<Action> set_extra_config =
+      ActionBuilder("set_extra_config", &SetExtraConfig)
+      .Description("Change a extra configuration value on a table")
+      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
+      .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
+      .AddRequiredParameter({ kConfigNameArg, "Name of the configuration" })
+      .AddRequiredParameter({ kConfigValueArg, "New value for the configuration" })
+      .Build();
+
+  unique_ptr<Action> get_extra_configs =
+      ActionBuilder("get_extra_configs", &GetExtraConfigs)
+      .Description("Get the extra configuration properties for a table")
+      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
+      .AddRequiredParameter({ kTableNameArg,
+                              "Name of the table for which to get extra configurations" })
+      .AddOptionalParameter("config_names")
+      .Build();
+
   return ModeBuilder("table")
       .Description("Operate on Kudu tables")
       .AddAction(std::move(delete_table))
@@ -523,6 +587,8 @@ unique_ptr<Mode> BuildTableMode() {
       .AddAction(std::move(rename_table))
       .AddAction(std::move(scan_table))
       .AddAction(std::move(copy_table))
+      .AddAction(std::move(set_extra_config))
+      .AddAction(std::move(get_extra_configs))
       .Build();
 }
 


[kudu] 03/03: [docs] Ubuntu 18.04 is a supported platform

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 02fa2d639f7ab8f50af21cd23967da42a4f7cde0
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Jun 20 01:23:53 2019 -0700

    [docs] Ubuntu 18.04 is a supported platform
    
    Once KUDU-2427 and KUDU-2871 are resolved, Kudu can run at
    Ubuntu 18.04 LTS (bionic).
    
    Change-Id: I633dc0a525bb1aac77a58755b535831ce436b79b
    Reviewed-on: http://gerrit.cloudera.org:8080/13685
    Reviewed-by: Grant Henke <gr...@apache.org>
    Tested-by: Grant Henke <gr...@apache.org>
---
 docs/installation.adoc | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/installation.adoc b/docs/installation.adoc
index 710423b..a65792b 100644
--- a/docs/installation.adoc
+++ b/docs/installation.adoc
@@ -44,8 +44,8 @@ only tolerate a single failure; two-master deployments cannot tolerate any failu
 
 .Operating System Requirements
 Linux::
-    - RHEL 6, RHEL 7, CentOS 6, CentOS 7, Ubuntu 14.04 (Trusty), Ubuntu 16.04 (Xenial), Debian 8
-    (Jessie), or SLES 12.
+    - RHEL 6, RHEL 7, CentOS 6, CentOS 7, Ubuntu 14.04 (trusty), Ubuntu 16.04 (xenial),
+      Ubuntu 18.04 (bionic), Debian 8 (Jessie), or SLES 12.
     - A kernel and filesystem that support _hole punching_. Hole punching is the use of the
       `fallocate(2)` system call with the `FALLOC_FL_PUNCH_HOLE` option set. See
       link:troubleshooting.html#req_hole_punching[troubleshooting hole punching] for more