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

[kudu] 02/02: KUDU-2514 Part 2: Supports setting history_max_age_sec for the specified table

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

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

commit 7b68bbd6f4696471077f876917724ab5262e9eb8
Author: oclarms <oc...@gmail.com>
AuthorDate: Mon Jun 10 11:02:53 2019 +0800

    KUDU-2514 Part 2: Supports setting history_max_age_sec for the specified table
    
    In the previous patch, we implemented the framework of extra configuration
    properties that are customizable on a per-table basis. So, we can add custom
    configurations to the specified table.
    
    In this patch, I also added the implementation of the C++ API.
    
    Change-Id: Iaac649154ae3c60785736ab246074c4dcb5b8987
    Reviewed-on: http://gerrit.cloudera.org:8080/13573
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/client/client-internal.cc                 | 15 ++++-
 src/kudu/client/client-internal.h                  | 15 +++--
 src/kudu/client/client-test.cc                     | 66 ++++++++++++++++++++--
 src/kudu/client/client.cc                          | 28 ++++++++-
 src/kudu/client/client.h                           | 27 ++++++++-
 src/kudu/client/table-internal.cc                  |  8 ++-
 src/kudu/client/table-internal.h                   |  6 +-
 src/kudu/client/table_alterer-internal.cc          |  8 ++-
 src/kudu/client/table_alterer-internal.h           |  3 +
 src/kudu/client/table_creator-internal.h           |  3 +
 .../integration-tests/tablet_history_gc-itest.cc   | 36 +++++++++---
 src/kudu/tablet/tablet.cc                          |  9 ++-
 12 files changed, 194 insertions(+), 30 deletions(-)

diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index fc310c0..3e1d8e1 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -22,6 +22,7 @@
 #include <cstdlib>
 #include <functional>
 #include <limits>
+#include <map>
 #include <memory>
 #include <mutex>
 #include <ostream>
@@ -72,6 +73,7 @@ DECLARE_int32(dns_resolver_max_threads_num);
 DECLARE_uint32(dns_resolver_cache_capacity_mb);
 DECLARE_uint32(dns_resolver_cache_ttl_sec);
 
+using std::map;
 using std::pair;
 using std::set;
 using std::shared_ptr;
@@ -461,6 +463,7 @@ Status KuduClient::Data::OpenTable(KuduClient* client,
   string table_name;
   int num_replicas;
   PartitionSchema partition_schema;
+  map<string, string> extra_configs;
   MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout_;
   RETURN_NOT_OK(GetTableSchema(client,
                                deadline,
@@ -469,7 +472,8 @@ Status KuduClient::Data::OpenTable(KuduClient* client,
                                &partition_schema,
                                &table_id,
                                &table_name,
-                               &num_replicas));
+                               &num_replicas,
+                               &extra_configs));
 
   // When the table name is specified, use the caller-provided table name.
   // This reduces surprises, e.g., when the HMS integration is on and table
@@ -482,7 +486,7 @@ Status KuduClient::Data::OpenTable(KuduClient* client,
   //                   map to reuse KuduTable instances.
   table->reset(new KuduTable(client->shared_from_this(),
                              effective_table_name, table_id, num_replicas,
-                             schema, partition_schema));
+                             schema, partition_schema, extra_configs));
 
   // When opening a table, clear the existing cached non-covered range entries.
   // This avoids surprises where a new table instance won't be able to see the
@@ -499,7 +503,8 @@ Status KuduClient::Data::GetTableSchema(KuduClient* client,
                                         PartitionSchema* partition_schema,
                                         std::string* table_id,
                                         std::string* table_name,
-                                        int* num_replicas) {
+                                        int* num_replicas,
+                                        map<string, string>* extra_configs) {
   GetTableSchemaRequestPB req;
   GetTableSchemaResponsePB resp;
 
@@ -539,6 +544,10 @@ Status KuduClient::Data::GetTableSchema(KuduClient* client,
   if (num_replicas) {
     *num_replicas = resp.num_replicas();
   }
+  if (extra_configs) {
+    map<string, string> result(resp.extra_configs().begin(), resp.extra_configs().end());
+    *extra_configs = std::move(result);
+  }
   // Cache the authz token if the response came with one. It might not have one
   // if running against an older master that does not support authz tokens.
   if (resp.has_authz_token()) {
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index 79ffd79..be9d0ed 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -20,6 +20,7 @@
 #include <cmath>
 #include <cstdint>
 #include <cstdlib>
+#include <map>
 #include <memory>
 #include <set>
 #include <string>
@@ -144,9 +145,14 @@ class KuduClient::Data {
                    const master::TableIdentifierPB& table_identifier,
                    client::sp::shared_ptr<KuduTable>* table);
 
-  // Retrieve table information about the table identified by 'table_id_or_name'.
-  // 'table_id_or_name' should contain a table id or name as 'identifier_type'
-  // is ID or NAME, respectively.
+  // Get the table information identified by 'table_identifier'.
+  // The table information (they can be null):
+  //   'schema'           The schema for the table.
+  //   'partition_schema' The partition schema for the table.
+  //   'table_id'         The table unique id.
+  //   'table_name'       The table unique name.
+  //   'num_replicas'     The table replication factor.
+  //   'extra_configs'    The table's extra configuration properties.
   Status GetTableSchema(KuduClient* client,
                         const MonoTime& deadline,
                         const master::TableIdentifierPB& table,
@@ -154,7 +160,8 @@ class KuduClient::Data {
                         PartitionSchema* partition_schema,
                         std::string* table_id,
                         std::string* table_name,
-                        int* num_replicas);
+                        int* num_replicas,
+                        std::map<std::string, std::string>* extra_configs);
 
   Status InitLocalHostNames();
 
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 8547b89..10c3223 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -1282,7 +1282,6 @@ TEST_F(ClientTest, TestInvalidPredicates) {
             "32-bit signed integer column 'int_val'", s.ToString());
 }
 
-
 // Check that the tserver proxy is reset on close, even for empty tables.
 TEST_F(ClientTest, TestScanCloseProxy) {
   const string kEmptyTable = "TestScanCloseProxy";
@@ -1901,7 +1900,6 @@ TEST_F(ClientTest, TestSwappedRangeBounds) {
                           "range partition lower bound must be less than the upper bound");
 }
 
-
 TEST_F(ClientTest, TestEqualRangeBounds) {
   KuduSchemaBuilder builder;
   KuduSchema schema;
@@ -3605,7 +3603,6 @@ TEST_F(ClientTest, TestUpsert) {
   }
 }
 
-
 TEST_F(ClientTest, TestWriteWithBadColumn) {
   shared_ptr<KuduTable> table;
   ASSERT_OK(client_->OpenTable(kTableName, &table));
@@ -3895,6 +3892,43 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_EQ(16 * 1024 * 1024, col_schema.attributes().cfile_block_size);
   }
 
+  // Test altering extra configuration properties.
+  // 1. Alter history max age second to 3600.
+  {
+    map<string, string> extra_configs;
+    extra_configs["kudu.table.history_max_age_sec"] = "3600";
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->AlterExtraConfig(extra_configs);
+    ASSERT_OK(table_alterer->Alter());
+    ASSERT_EQ(9, tablet_replica->tablet()->metadata()->schema_version());
+    ASSERT_NE(boost::none, tablet_replica->tablet()->metadata()->extra_config());
+    ASSERT_TRUE(tablet_replica->tablet()->metadata()->extra_config()->has_history_max_age_sec());
+    ASSERT_EQ(3600, tablet_replica->tablet()->metadata()->extra_config()->history_max_age_sec());
+  }
+  // 2. Alter history max age second to 7200.
+  {
+    map<string, string> extra_configs;
+    extra_configs["kudu.table.history_max_age_sec"] = "7200";
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->AlterExtraConfig(extra_configs);
+    ASSERT_OK(table_alterer->Alter());
+    ASSERT_EQ(10, tablet_replica->tablet()->metadata()->schema_version());
+    ASSERT_NE(boost::none, tablet_replica->tablet()->metadata()->extra_config());
+    ASSERT_TRUE(tablet_replica->tablet()->metadata()->extra_config()->has_history_max_age_sec());
+    ASSERT_EQ(7200, tablet_replica->tablet()->metadata()->extra_config()->history_max_age_sec());
+  }
+  // 3. Reset history max age second to default.
+  {
+    map<string, string> extra_configs;
+    extra_configs["kudu.table.history_max_age_sec"] = "";
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->AlterExtraConfig(extra_configs);
+    ASSERT_OK(table_alterer->Alter());
+    ASSERT_EQ(11, tablet_replica->tablet()->metadata()->schema_version());
+    ASSERT_NE(boost::none, tablet_replica->tablet()->metadata()->extra_config());
+    ASSERT_FALSE(tablet_replica->tablet()->metadata()->extra_config()->has_history_max_age_sec());
+  }
+
   // Test changing a table name.
   {
     const char *kRenamedTableName = "RenamedTable";
@@ -3902,7 +3936,7 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_OK(table_alterer
               ->RenameTo(kRenamedTableName)
               ->Alter());
-    ASSERT_EQ(9, tablet_replica->tablet()->metadata()->schema_version());
+    ASSERT_EQ(12, tablet_replica->tablet()->metadata()->schema_version());
     ASSERT_EQ(kRenamedTableName, tablet_replica->tablet()->metadata()->table_name());
 
     CatalogManager *catalog_manager = cluster_->mini_master()->master()->catalog_manager();
@@ -4585,6 +4619,30 @@ TEST_F(ClientTest, TestCreateTableWithTooLongColumnName) {
                      "longer than maximum permitted length 256");
 }
 
+TEST_F(ClientTest, TestCreateTableWithExtraConfigs) {
+  string table_name = "extra_configs";
+  {
+    map<string, string> extra_configs;
+    extra_configs["kudu.table.history_max_age_sec"] = "7200";
+    unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+    Status s = table_creator->table_name(table_name)
+        .set_range_partition_columns({"key"})
+        .schema(&schema_)
+        .num_replicas(1)
+        .extra_configs(extra_configs)
+        .Create();
+    ASSERT_TRUE(s.ok());
+  }
+
+  {
+    shared_ptr<KuduTable> table;
+    ASSERT_OK(client_->OpenTable(table_name, &table));
+    map<string, string> extra_configs = table->extra_configs();
+    ASSERT_TRUE(ContainsKey(extra_configs, "kudu.table.history_max_age_sec"));
+    ASSERT_EQ("7200", extra_configs["kudu.table.history_max_age_sec"]);
+  }
+}
+
 // Test trying to insert a row with an encoded key that is too large.
 TEST_F(ClientTest, TestInsertTooLongEncodedPrimaryKey) {
   const string kLongValue(10000, 'x');
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 42ce211..cc4ee3f 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -18,6 +18,7 @@
 #include "kudu/client/client.h"
 
 #include <cstdlib>
+#include <map>
 #include <memory>
 #include <mutex>
 #include <ostream>
@@ -124,6 +125,7 @@ using kudu::rpc::MessengerBuilder;
 using kudu::rpc::RpcController;
 using kudu::rpc::UserCredentials;
 using kudu::tserver::ScanResponsePB;
+using std::map;
 using std::set;
 using std::string;
 using std::unique_ptr;
@@ -436,7 +438,8 @@ Status KuduClient::GetTableSchema(const string& table_name,
                                nullptr, // partition schema
                                nullptr, // table id
                                nullptr, // table name
-                               nullptr); // number of replicas
+                               nullptr, // number of replicas
+                               nullptr); // extra configs
 }
 
 Status KuduClient::ListTabletServers(vector<KuduTabletServer*>* tablet_servers) {
@@ -721,6 +724,11 @@ KuduTableCreator& KuduTableCreator::num_replicas(int num_replicas) {
   return *this;
 }
 
+KuduTableCreator& KuduTableCreator::extra_configs(const map<string, string>& extra_configs) {
+  data_->extra_configs_ = extra_configs;
+  return *this;
+}
+
 KuduTableCreator& KuduTableCreator::timeout(const MonoDelta& timeout) {
   data_->timeout_ = timeout;
   return *this;
@@ -752,6 +760,10 @@ Status KuduTableCreator::Create() {
   if (data_->num_replicas_ != boost::none) {
     req.set_num_replicas(data_->num_replicas_.get());
   }
+  if (data_->extra_configs_) {
+    req.mutable_extra_configs()->insert(data_->extra_configs_->begin(),
+                                        data_->extra_configs_->end());
+  }
   RETURN_NOT_OK_PREPEND(SchemaToPB(*data_->schema_->schema_, req.mutable_schema(),
                                    SCHEMA_PB_WITHOUT_WRITE_DEFAULT),
                         "Invalid schema");
@@ -821,9 +833,10 @@ KuduTable::KuduTable(const shared_ptr<KuduClient>& client,
                      const string& id,
                      int num_replicas,
                      const KuduSchema& schema,
-                     const PartitionSchema& partition_schema)
+                     const PartitionSchema& partition_schema,
+                     const map<string, string>& extra_configs)
   : data_(new KuduTable::Data(client, name, id, num_replicas,
-                              schema, partition_schema)) {
+                              schema, partition_schema, extra_configs)) {
 }
 
 KuduTable::~KuduTable() {
@@ -870,6 +883,10 @@ const PartitionSchema& KuduTable::partition_schema() const {
   return data_->partition_schema_;
 }
 
+const map<string, string>& KuduTable::extra_configs() const {
+  return data_->extra_configs_;
+}
+
 KuduPredicate* KuduTable::NewComparisonPredicate(const Slice& col_name,
                                                  KuduPredicate::ComparisonOp op,
                                                  KuduValue* value) {
@@ -1173,6 +1190,11 @@ KuduTableAlterer* KuduTableAlterer::DropRangePartition(
   return this;
 }
 
+KuduTableAlterer* KuduTableAlterer::AlterExtraConfig(const map<string, string>& extra_configs) {
+  data_->new_extra_configs_ = extra_configs;
+  return this;
+}
+
 KuduTableAlterer* KuduTableAlterer::timeout(const MonoDelta& timeout) {
   data_->timeout_ = timeout;
   return this;
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index fb5c0c0..2ef8da5 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -28,6 +28,7 @@
 #include <stdint.h>
 
 #include <cstddef>
+#include <map>
 #include <string>
 #include <vector>
 
@@ -873,6 +874,15 @@ class KUDU_EXPORT KuduTableCreator {
   /// @return Reference to the modified table creator.
   KuduTableCreator& num_replicas(int n_replicas);
 
+  /// Sets the table's extra configuration properties.
+  ///
+  /// If the value of the kv pair is empty, the property will be ignored.
+  ///
+  /// @param [in] extra_configs
+  ///   The table's extra configuration properties.
+  /// @return Reference to the modified table creator.
+  KuduTableCreator& extra_configs(const std::map<std::string, std::string>& extra_configs);
+
   /// Set the timeout for the table creation operation.
   ///
   /// This includes any waiting after the create has been submitted
@@ -1067,6 +1077,9 @@ class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
   /// @return The partition schema for the table.
   const PartitionSchema& partition_schema() const;
 
+  /// @return The table's extra configuration properties.
+  const std::map<std::string, std::string>& extra_configs() const;
+
   /// @cond PRIVATE_API
 
   /// List the partitions of this table in 'partitions'. This operation may
@@ -1093,7 +1106,8 @@ class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
             const std::string& id,
             int num_replicas,
             const KuduSchema& schema,
-            const PartitionSchema& partition_schema);
+            const PartitionSchema& partition_schema,
+            const std::map<std::string, std::string>& extra_configs);
 
   // Owned.
   Data* data_;
@@ -1223,6 +1237,17 @@ class KUDU_EXPORT KuduTableAlterer {
       KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
       KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
 
+  /// Change the table's extra configuration properties.
+  ///
+  /// @note These configuration properties will be merged into existing configuration properties.
+  ///
+  /// @note If the value of the kv pair is empty, the property will be unset.
+  ///
+  /// @param [in] extra_configs
+  ///   The table's extra configuration properties.
+  /// @return Raw pointer to this alterer object.
+  KuduTableAlterer* AlterExtraConfig(const std::map<std::string, std::string>& extra_configs);
+
   /// Set a timeout for the alteration operation.
   ///
   /// This includes any waiting after the alter has been submitted
diff --git a/src/kudu/client/table-internal.cc b/src/kudu/client/table-internal.cc
index d7ab570..7114ef8 100644
--- a/src/kudu/client/table-internal.cc
+++ b/src/kudu/client/table-internal.cc
@@ -17,9 +17,11 @@
 
 #include "kudu/client/table-internal.h"
 
+#include <map>
 #include <string>
 #include <utility>
 
+using std::map;
 using std::string;
 
 namespace kudu {
@@ -32,13 +34,15 @@ KuduTable::Data::Data(shared_ptr<KuduClient> client,
                       string id,
                       int num_replicas,
                       const KuduSchema& schema,
-                      PartitionSchema partition_schema)
+                      PartitionSchema partition_schema,
+                      map<string, string> extra_configs)
     : client_(std::move(client)),
       name_(std::move(name)),
       id_(std::move(id)),
       num_replicas_(num_replicas),
       schema_(schema),
-      partition_schema_(std::move(partition_schema)) {
+      partition_schema_(std::move(partition_schema)),
+      extra_configs_(std::move(extra_configs)) {
 }
 
 KuduTable::Data::~Data() {
diff --git a/src/kudu/client/table-internal.h b/src/kudu/client/table-internal.h
index f1d9cfd..c1de109 100644
--- a/src/kudu/client/table-internal.h
+++ b/src/kudu/client/table-internal.h
@@ -17,6 +17,7 @@
 #ifndef KUDU_CLIENT_TABLE_INTERNAL_H
 #define KUDU_CLIENT_TABLE_INTERNAL_H
 
+#include <map>
 #include <string>
 
 #include "kudu/client/client.h"
@@ -42,7 +43,8 @@ class KuduTable::Data {
        std::string id,
        int num_replicas,
        const KuduSchema& schema,
-       PartitionSchema partition_schema);
+       PartitionSchema partition_schema,
+       std::map<std::string, std::string> extra_configs);
   ~Data();
 
   template<class PredicateMakerFunc>
@@ -73,6 +75,8 @@ class KuduTable::Data {
   const KuduSchema schema_;
   const PartitionSchema partition_schema_;
 
+  const std::map<std::string, std::string> extra_configs_;
+
  private:
   DISALLOW_COPY_AND_ASSIGN(Data);
 };
diff --git a/src/kudu/client/table_alterer-internal.cc b/src/kudu/client/table_alterer-internal.cc
index 56bfcd4..7b5302d 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -56,16 +56,20 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
     return status_;
   }
 
-  if (!rename_to_.is_initialized() && steps_.empty()) {
+  if (!rename_to_ && !new_extra_configs_ && steps_.empty()) {
     return Status::InvalidArgument("No alter steps provided");
   }
 
   req->Clear();
   req->set_modify_external_catalogs(modify_external_catalogs_);
   req->mutable_table()->set_table_name(table_name_);
-  if (rename_to_.is_initialized()) {
+  if (rename_to_) {
     req->set_new_table_name(rename_to_.get());
   }
+  if (new_extra_configs_) {
+    req->mutable_new_extra_configs()->insert(new_extra_configs_->begin(),
+                                             new_extra_configs_->end());
+  }
 
   if (schema_ != nullptr) {
     RETURN_NOT_OK(SchemaToPB(*schema_, req->mutable_schema(),
diff --git a/src/kudu/client/table_alterer-internal.h b/src/kudu/client/table_alterer-internal.h
index 353b79d..df480a7 100644
--- a/src/kudu/client/table_alterer-internal.h
+++ b/src/kudu/client/table_alterer-internal.h
@@ -17,6 +17,7 @@
 #ifndef KUDU_CLIENT_TABLE_ALTERER_INTERNAL_H
 #define KUDU_CLIENT_TABLE_ALTERER_INTERNAL_H
 
+#include <map>
 #include <memory>
 #include <string>
 #include <vector>
@@ -71,6 +72,8 @@ class KuduTableAlterer::Data {
 
   boost::optional<std::string> rename_to_;
 
+  boost::optional<std::map<std::string, std::string>> new_extra_configs_;
+
   // Set to true if there are alter partition steps.
   bool has_alter_partitioning_steps = false;
 
diff --git a/src/kudu/client/table_creator-internal.h b/src/kudu/client/table_creator-internal.h
index 4923051..f631845 100644
--- a/src/kudu/client/table_creator-internal.h
+++ b/src/kudu/client/table_creator-internal.h
@@ -17,6 +17,7 @@
 #ifndef KUDU_CLIENT_TABLE_CREATOR_INTERNAL_H
 #define KUDU_CLIENT_TABLE_CREATOR_INTERNAL_H
 
+#include <map>
 #include <memory>
 #include <string>
 #include <vector>
@@ -61,6 +62,8 @@ class KuduTableCreator::Data {
 
   boost::optional<int> num_replicas_;
 
+  boost::optional<std::map<std::string, std::string>> extra_configs_;
+
   MonoDelta timeout_;
 
   bool wait_;
diff --git a/src/kudu/integration-tests/tablet_history_gc-itest.cc b/src/kudu/integration-tests/tablet_history_gc-itest.cc
index c3c97fb..e2e2371 100644
--- a/src/kudu/integration-tests/tablet_history_gc-itest.cc
+++ b/src/kudu/integration-tests/tablet_history_gc-itest.cc
@@ -71,15 +71,18 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
+using kudu::client::KuduClient;
 using kudu::client::KuduScanner;
 using kudu::client::KuduSession;
 using kudu::client::KuduTable;
+using kudu::client::KuduTableAlterer;
 using kudu::client::sp::shared_ptr;
 using kudu::clock::HybridClock;
 using kudu::tablet::Tablet;
 using kudu::tablet::TabletReplica;
 using kudu::tserver::MiniTabletServer;
 using kudu::tserver::TabletServer;
+using std::map;
 using std::string;
 using std::unique_ptr;
 using std::vector;
@@ -122,19 +125,36 @@ TEST_F(TabletHistoryGcITest, TestSnapshotScanBeforeAHM) {
   TestWorkload workload(cluster_.get());
   workload.Setup();
 
+  auto open_scanner_func = [](KuduClient* client) -> Status {
+    shared_ptr<KuduTable> table;
+    RETURN_NOT_OK(client->OpenTable(TestWorkload::kDefaultTableName, &table));
+    KuduScanner scanner(table.get());
+    RETURN_NOT_OK(scanner.SetReadMode(KuduScanner::READ_AT_SNAPSHOT));
+    return scanner.Open();
+  };
+
   // When the tablet history max age is set to 0, it's not possible to do a
   // snapshot scan without a timestamp because it's illegal to open a snapshot
   // prior to the AHM. When a snapshot timestamp is not specified, we decide on
   // the timestamp of the snapshot before checking that it's lower than the
   // current AHM. This test verifies that scans prior to the AHM are rejected.
-  shared_ptr<KuduTable> table;
-  ASSERT_OK(client_->OpenTable(TestWorkload::kDefaultTableName, &table));
-  KuduScanner scanner(table.get());
-  ASSERT_OK(scanner.SetReadMode(KuduScanner::READ_AT_SNAPSHOT));
-  Status s = scanner.Open();
-  ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(),
-                      "snapshot scan end timestamp is earlier than the ancient history mark");
+  {
+    Status s = open_scanner_func(client_.get());
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "snapshot scan end timestamp is earlier than the ancient history mark");
+  }
+
+  // Alter the table's history max age to 3600. We can scan it.
+  {
+    map<string, string> extra_configs;
+    extra_configs["kudu.table.history_max_age_sec"] = "3600";
+    unique_ptr<KuduTableAlterer> table_alterer(
+        client_->NewTableAlterer(TestWorkload::kDefaultTableName));
+    table_alterer->AlterExtraConfig(extra_configs);
+    ASSERT_OK(table_alterer->Alter());
+    ASSERT_OK(open_scanner_func(client_.get()));
+  }
 }
 
 // Check that the maintenance manager op to delete undo deltas actually deletes them.
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 1506b6a..04e312c 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1078,14 +1078,19 @@ Status Tablet::DoMajorDeltaCompaction(const vector<ColumnId>& col_ids,
 }
 
 bool Tablet::GetTabletAncientHistoryMark(Timestamp* ancient_history_mark) const {
+  int32_t tablet_history_max_age_sec = FLAGS_tablet_history_max_age_sec;
+  if (metadata_->extra_config() && metadata_->extra_config()->has_history_max_age_sec()) {
+    // Override the global configuration with the configuration of the table
+    tablet_history_max_age_sec = metadata_->extra_config()->history_max_age_sec();
+  }
   // We currently only support history GC through a fully-instantiated tablet
   // when using the HybridClock, since we can calculate the age of a mutation.
-  if (!clock_->HasPhysicalComponent() || FLAGS_tablet_history_max_age_sec < 0) {
+  if (!clock_->HasPhysicalComponent() || tablet_history_max_age_sec < 0) {
     return false;
   }
   Timestamp now = clock_->Now();
   uint64_t now_micros = HybridClock::GetPhysicalValueMicros(now);
-  uint64_t max_age_micros = FLAGS_tablet_history_max_age_sec * 1000000ULL;
+  uint64_t max_age_micros = tablet_history_max_age_sec * 1000000ULL;
   // Ensure that the AHM calculation doesn't underflow when
   // '--tablet_history_max_age_sec' is set to a very high value.
   if (max_age_micros <= now_micros) {