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/08 18:28:39 UTC

[kudu] branch master updated (9dfe8f8 -> ecd3612)

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 9dfe8f8  KUDU-2809 (5/6): add diff scan support to fuzz-itest
     new b0ee399  [c++] Support table rename between scan token creation and rehydration
     new ecd3612  KUDU-2514 Part 1: Support extra config for table.

The 2 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:
 .../org/apache/kudu/client/AlterTableOptions.java  | 15 +++++
 .../org/apache/kudu/client/AsyncKuduClient.java    |  5 +-
 .../org/apache/kudu/client/CreateTableOptions.java | 14 ++++
 .../apache/kudu/client/GetTableSchemaRequest.java  |  3 +-
 .../apache/kudu/client/GetTableSchemaResponse.java | 16 ++++-
 .../java/org/apache/kudu/client/KuduTable.java     | 15 ++++-
 .../org/apache/kudu/client/TestAlterTable.java     | 42 ++++++++++++
 .../apache/kudu/client/TestAsyncKuduClient.java    |  2 +-
 .../org/apache/kudu/client/TestKeyEncoding.java    |  6 +-
 .../org/apache/kudu/client/TestKuduClient.java     | 22 +++++++
 src/kudu/client/client-internal.cc                 | 53 +++++++++++++++-
 src/kudu/client/client-internal.h                  | 11 +++-
 src/kudu/client/client.cc                          | 35 +++-------
 src/kudu/client/scan_token-internal.cc             | 16 ++++-
 src/kudu/client/scan_token-test.cc                 | 48 ++++++++++++++
 src/kudu/common/common.proto                       |  8 +++
 src/kudu/common/wire_protocol.cc                   | 45 +++++++++++++
 src/kudu/common/wire_protocol.h                    | 15 +++++
 src/kudu/master/catalog_manager.cc                 | 44 ++++++++++---
 src/kudu/master/catalog_manager.h                  |  4 +-
 src/kudu/master/master.proto                       | 10 +++
 src/kudu/master/master_path_handlers.cc            | 11 ++++
 src/kudu/master/sys_catalog.cc                     |  1 +
 src/kudu/tablet/metadata.proto                     |  3 +
 src/kudu/tablet/tablet-harness.h                   |  1 +
 src/kudu/tablet/tablet-test-util.h                 |  6 +-
 src/kudu/tablet/tablet.cc                          |  3 +
 src/kudu/tablet/tablet_bootstrap-test.cc           |  1 +
 src/kudu/tablet/tablet_metadata-test.cc            | 74 +++++++++++++++++-----
 src/kudu/tablet/tablet_metadata.cc                 | 31 ++++++++-
 src/kudu/tablet/tablet_metadata.h                  | 14 +++-
 .../tablet/transactions/alter_schema_transaction.h |  8 +++
 src/kudu/tools/kudu-tool-test.cc                   |  2 +
 src/kudu/tserver/mini_tablet_server.cc             |  3 +-
 src/kudu/tserver/tablet_copy_client.cc             |  2 +
 src/kudu/tserver/tablet_server-test.cc             |  2 +-
 src/kudu/tserver/tablet_service.cc                 | 18 +++---
 src/kudu/tserver/ts_tablet_manager-test.cc         | 43 +++++++++----
 src/kudu/tserver/ts_tablet_manager.cc              |  3 +
 src/kudu/tserver/ts_tablet_manager.h               |  5 ++
 src/kudu/tserver/tserver_admin.proto               |  4 ++
 www/table.mustache                                 |  3 +
 42 files changed, 575 insertions(+), 92 deletions(-)


[kudu] 01/02: [c++] Support table rename between scan token creation and rehydration

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 b0ee39962d0b9766a7d9de9b7207524495319631
Author: Will Berkeley <wd...@gmail.com>
AuthorDate: Wed Jun 5 13:38:33 2019 -0700

    [c++] Support table rename between scan token creation and rehydration
    
    Previously, if a scan token was created against table 'foo', and table
    'foo' was renamed to table 'bar', rehydrating the scan token into a
    scanner would either
    - fail if there is no longer a table 'foo', or
    - attempt to scan another table if that table had been renamed to 'foo'.
    This patch alters how the C++ client manages scan tokens. It prefers to
    use the table id to identify a table, rather than a table name. This
    eliminates the above two problems.
    
    This is a follow up to the Java client patch for the same (and that
    patch includes the necessary protobuf changes).
    
    Change-Id: Ib4d48513dff67012f26a99877b168d777d3049fd
    Reviewed-on: http://gerrit.cloudera.org:8080/13521
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
    Reviewed-by: Mike Percy <mp...@apache.org>
---
 src/kudu/client/client-internal.cc     | 53 ++++++++++++++++++++++++++++++++--
 src/kudu/client/client-internal.h      | 11 ++++++-
 src/kudu/client/client.cc              | 35 ++++++----------------
 src/kudu/client/scan_token-internal.cc | 16 +++++++++-
 src/kudu/client/scan_token-test.cc     | 48 ++++++++++++++++++++++++++++++
 5 files changed, 132 insertions(+), 31 deletions(-)

diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index 09e42bb..fc310c0 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -453,17 +453,61 @@ bool KuduClient::Data::FetchCachedAuthzToken(const string& table_id, SignedToken
   return authz_token_cache_.Fetch(table_id, token);
 }
 
+Status KuduClient::Data::OpenTable(KuduClient* client,
+                                   const TableIdentifierPB& table_identifier,
+                                   client::sp::shared_ptr<KuduTable>* table) {
+  KuduSchema schema;
+  string table_id;
+  string table_name;
+  int num_replicas;
+  PartitionSchema partition_schema;
+  MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout_;
+  RETURN_NOT_OK(GetTableSchema(client,
+                               deadline,
+                               table_identifier,
+                               &schema,
+                               &partition_schema,
+                               &table_id,
+                               &table_name,
+                               &num_replicas));
+
+  // 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
+  // names are case-insensitive.
+  string effective_table_name = table_identifier.has_table_name() ?
+      table_identifier.table_name() :
+      table_name;
+
+  // TODO(wdberkeley): In the future, probably will look up the table in some
+  //                   map to reuse KuduTable instances.
+  table->reset(new KuduTable(client->shared_from_this(),
+                             effective_table_name, table_id, num_replicas,
+                             schema, partition_schema));
+
+  // 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
+  // current range partitions of a table for up to the ttl.
+  meta_cache_->ClearNonCoveredRangeEntries(table_id);
+
+  return Status::OK();
+}
+
 Status KuduClient::Data::GetTableSchema(KuduClient* client,
-                                        const string& table_name,
                                         const MonoTime& deadline,
+                                        const TableIdentifierPB& table,
                                         KuduSchema* schema,
                                         PartitionSchema* partition_schema,
-                                        string* table_id,
+                                        std::string* table_id,
+                                        std::string* table_name,
                                         int* num_replicas) {
   GetTableSchemaRequestPB req;
   GetTableSchemaResponsePB resp;
 
-  req.mutable_table()->set_table_name(table_name);
+  if (table.has_table_id()) {
+    req.mutable_table()->set_table_id(table.table_id());
+  } else {
+    req.mutable_table()->set_table_name(table.table_name());
+  }
   Synchronizer sync;
   AsyncLeaderMasterRpc<GetTableSchemaRequestPB, GetTableSchemaResponsePB> rpc(
       deadline, client, BackoffType::EXPONENTIAL, req, &resp,
@@ -486,6 +530,9 @@ Status KuduClient::Data::GetTableSchema(KuduClient* client,
   if (partition_schema) {
     *partition_schema = std::move(new_partition_schema);
   }
+  if (table_name) {
+    *table_name = resp.table_name();
+  }
   if (table_id) {
     *table_id = resp.table_id();
   }
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index 2098bda..79ffd79 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -139,12 +139,21 @@ class KuduClient::Data {
                                    master::TableIdentifierPB table,
                                    const MonoTime& deadline);
 
+  // Open the table identified by 'table_identifier'.
+  Status OpenTable(KuduClient* client,
+                   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.
   Status GetTableSchema(KuduClient* client,
-                        const std::string& table_name,
                         const MonoTime& deadline,
+                        const master::TableIdentifierPB& table,
                         KuduSchema* schema,
                         PartitionSchema* partition_schema,
                         std::string* table_id,
+                        std::string* table_name,
                         int* num_replicas);
 
   Status InitLocalHostNames();
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 8cde561..42ce211 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -427,12 +427,15 @@ Status KuduClient::IsAlterTableInProgress(const string& table_name,
 Status KuduClient::GetTableSchema(const string& table_name,
                                   KuduSchema* schema) {
   MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
+  TableIdentifierPB table;
+  table.set_table_name(table_name);
   return data_->GetTableSchema(this,
-                               table_name,
                                deadline,
+                               table,
                                schema,
                                nullptr, // partition schema
                                nullptr, // table id
+                               nullptr, // table name
                                nullptr); // number of replicas
 }
 
@@ -500,31 +503,11 @@ Status KuduClient::TableExists(const string& table_name, bool* exists) {
 
 Status KuduClient::OpenTable(const string& table_name,
                              shared_ptr<KuduTable>* table) {
-  KuduSchema schema;
-  string table_id;
-  int num_replicas;
-  PartitionSchema partition_schema;
-  MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
-  RETURN_NOT_OK(data_->GetTableSchema(this,
-                                      table_name,
-                                      deadline,
-                                      &schema,
-                                      &partition_schema,
-                                      &table_id,
-                                      &num_replicas));
-
-  // TODO: in the future, probably will look up the table in some map to reuse
-  // KuduTable instances.
-  table->reset(new KuduTable(shared_from_this(),
-                             table_name, table_id, num_replicas,
-                             schema, partition_schema));
-
-  // 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
-  // current range partitions of a table for up to the ttl.
-  data_->meta_cache_->ClearNonCoveredRangeEntries(table_id);
-
-  return Status::OK();
+  TableIdentifierPB table_identifier;
+  table_identifier.set_table_name(table_name);
+  return data_->OpenTable(this,
+                          table_identifier,
+                          table);
 }
 
 shared_ptr<KuduSession> KuduClient::NewSession() {
diff --git a/src/kudu/client/scan_token-internal.cc b/src/kudu/client/scan_token-internal.cc
index 46bc270..86f73bb 100644
--- a/src/kudu/client/scan_token-internal.cc
+++ b/src/kudu/client/scan_token-internal.cc
@@ -48,6 +48,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/master/master.pb.h"
 #include "kudu/util/async_util.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -60,6 +61,9 @@ using std::vector;
 using strings::Substitute;
 
 namespace kudu {
+
+using master::TableIdentifierPB;
+
 namespace client {
 
 using internal::MetaCache;
@@ -103,8 +107,17 @@ Status KuduScanToken::Data::PBIntoScanner(KuduClient* client,
     }
   }
 
+  TableIdentifierPB table_identifier;
+  if (message.has_table_id()) {
+    table_identifier.set_table_id(message.table_id());
+  }
+  if (message.has_table_name()) {
+    table_identifier.set_table_name(message.table_name());
+  }
   sp::shared_ptr<KuduTable> table;
-  RETURN_NOT_OK(client->OpenTable(message.table_name(), &table));
+  RETURN_NOT_OK(client->data_->OpenTable(client,
+                                         table_identifier,
+                                         &table));
   Schema* schema = table->schema().schema_;
 
   unique_ptr<KuduScanner> scan_builder(new KuduScanner(table.get()));
@@ -230,6 +243,7 @@ Status KuduScanTokenBuilder::Data::Build(vector<KuduScanToken*>* tokens) {
 
   ScanTokenPB pb;
 
+  pb.set_table_id(table->id());
   pb.set_table_name(table->name());
   RETURN_NOT_OK(SchemaToColumnPBs(*configuration_.projection(), pb.mutable_projected_columns(),
                                   SCHEMA_PB_WITHOUT_STORAGE_ATTRIBUTES | SCHEMA_PB_WITHOUT_IDS));
diff --git a/src/kudu/client/scan_token-test.cc b/src/kudu/client/scan_token-test.cc
index fe94ff8..ebfb160 100644
--- a/src/kudu/client/scan_token-test.cc
+++ b/src/kudu/client/scan_token-test.cc
@@ -16,6 +16,7 @@
 // under the License.
 
 #include <atomic>
+#include <cstddef>
 #include <cstdint>
 #include <memory>
 #include <string>
@@ -27,6 +28,7 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
+#include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/client/client.pb.h"
 #include "kudu/client/scan_batch.h"
@@ -637,5 +639,51 @@ TEST_F(ScanTokenTest, TestConcurrentAlterTable) {
   delete scanner_ptr;
 }
 
+// Tests the results of creating scan tokens, renaming the table being
+// scanned, and then executing the scan tokens.
+TEST_F(ScanTokenTest, TestConcurrentRenameTable) {
+  const char* kTableName = "scan-token-rename";
+  // Create schema
+  KuduSchema schema;
+  {
+    KuduSchemaBuilder builder;
+    builder.AddColumn("key")->NotNull()->Type(KuduColumnSchema::INT64)->PrimaryKey();
+    builder.AddColumn("a")->NotNull()->Type(KuduColumnSchema::INT64);
+    ASSERT_OK(builder.Build(&schema));
+  }
+
+  // Create table
+  shared_ptr<KuduTable> table;
+  {
+    unique_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
+    ASSERT_OK(table_creator->table_name(kTableName)
+                            .schema(&schema)
+                            .set_range_partition_columns({})
+                            .num_replicas(1)
+                            .Create());
+    ASSERT_OK(client_->OpenTable(kTableName, &table));
+  }
+
+  vector<KuduScanToken*> tokens;
+  ASSERT_OK(KuduScanTokenBuilder(table.get()).Build(&tokens));
+  ASSERT_EQ(1, tokens.size());
+  unique_ptr<KuduScanToken> token(tokens[0]);
+
+  // Rename the table.
+  {
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->RenameTo("scan-token-rename-renamed");
+    ASSERT_OK(table_alterer->Alter());
+  }
+
+  KuduScanner* scanner_ptr;
+  ASSERT_OK(token->IntoKuduScanner(&scanner_ptr));
+  size_t row_count;
+  ASSERT_OK(CountRowsWithRetries(scanner_ptr, &row_count));
+  ASSERT_EQ(0, row_count);
+  delete scanner_ptr;
+}
+
+
 } // namespace client
 } // namespace kudu


[kudu] 02/02: KUDU-2514 Part 1: Support extra config for table.

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 ecd3612edd59c75cc119a8a234b6fe72a0e8c7fe
Author: oclarms <oc...@gmail.com>
AuthorDate: Mon May 20 20:42:18 2019 +0800

    KUDU-2514 Part 1: Support extra config for table.
    
    We have thousands of tables in the Kudu cluster. It's hard to set a
    uniform configuration for all tables. So, we support extra configuration
    properties that are customizable on a per-table basis.
    
    Note: In this patch, we implemented the framework, and the Java API. The
    configuration item only validate framework & Java API. It doesn't actually
    affect the tablet.
    
    Change-Id: I0514507dca95602a97e954d1db464b907e073aae
    Reviewed-on: http://gerrit.cloudera.org:8080/12468
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 .../org/apache/kudu/client/AlterTableOptions.java  | 15 +++++
 .../org/apache/kudu/client/AsyncKuduClient.java    |  5 +-
 .../org/apache/kudu/client/CreateTableOptions.java | 14 ++++
 .../apache/kudu/client/GetTableSchemaRequest.java  |  3 +-
 .../apache/kudu/client/GetTableSchemaResponse.java | 16 ++++-
 .../java/org/apache/kudu/client/KuduTable.java     | 15 ++++-
 .../org/apache/kudu/client/TestAlterTable.java     | 42 ++++++++++++
 .../apache/kudu/client/TestAsyncKuduClient.java    |  2 +-
 .../org/apache/kudu/client/TestKeyEncoding.java    |  6 +-
 .../org/apache/kudu/client/TestKuduClient.java     | 22 +++++++
 src/kudu/common/common.proto                       |  8 +++
 src/kudu/common/wire_protocol.cc                   | 45 +++++++++++++
 src/kudu/common/wire_protocol.h                    | 15 +++++
 src/kudu/master/catalog_manager.cc                 | 44 ++++++++++---
 src/kudu/master/catalog_manager.h                  |  4 +-
 src/kudu/master/master.proto                       | 10 +++
 src/kudu/master/master_path_handlers.cc            | 11 ++++
 src/kudu/master/sys_catalog.cc                     |  1 +
 src/kudu/tablet/metadata.proto                     |  3 +
 src/kudu/tablet/tablet-harness.h                   |  1 +
 src/kudu/tablet/tablet-test-util.h                 |  6 +-
 src/kudu/tablet/tablet.cc                          |  3 +
 src/kudu/tablet/tablet_bootstrap-test.cc           |  1 +
 src/kudu/tablet/tablet_metadata-test.cc            | 74 +++++++++++++++++-----
 src/kudu/tablet/tablet_metadata.cc                 | 31 ++++++++-
 src/kudu/tablet/tablet_metadata.h                  | 14 +++-
 .../tablet/transactions/alter_schema_transaction.h |  8 +++
 src/kudu/tools/kudu-tool-test.cc                   |  2 +
 src/kudu/tserver/mini_tablet_server.cc             |  3 +-
 src/kudu/tserver/tablet_copy_client.cc             |  2 +
 src/kudu/tserver/tablet_server-test.cc             |  2 +-
 src/kudu/tserver/tablet_service.cc                 | 18 +++---
 src/kudu/tserver/ts_tablet_manager-test.cc         | 43 +++++++++----
 src/kudu/tserver/ts_tablet_manager.cc              |  3 +
 src/kudu/tserver/ts_tablet_manager.h               |  5 ++
 src/kudu/tserver/tserver_admin.proto               |  4 ++
 www/table.mustache                                 |  3 +
 37 files changed, 443 insertions(+), 61 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
index 677df58..abf5538 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
@@ -18,6 +18,7 @@
 package org.apache.kudu.client;
 
 import java.util.EnumSet;
+import java.util.Map;
 
 import static org.apache.kudu.ColumnSchema.CompressionAlgorithm;
 import static org.apache.kudu.ColumnSchema.Encoding;
@@ -381,6 +382,20 @@ public class AlterTableOptions {
   }
 
   /**
+   * Change the table's extra configuration properties.
+   * These configuration properties will be merged into existing configuration properties.
+   *
+   * If the value of the kv pair is empty, the property will be unset.
+   *
+   * @param extraConfig the table's extra configuration properties
+   * @return this instance
+   */
+  public AlterTableOptions alterExtraConfigs(Map<String, String> extraConfig) {
+    pb.putAllNewExtraConfigs(extraConfig);
+    return this;
+  }
+
+  /**
    * Whether to wait for the table to be fully altered before this alter
    * operation is considered to be finished.
    * <p>
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index de5ae10..b8dac87 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -369,7 +369,7 @@ public class AsyncKuduClient implements AutoCloseable {
     this.channelFactory = b.createChannelFactory();
     this.masterAddresses = b.masterAddresses;
     this.masterTable = new KuduTable(this, MASTER_TABLE_NAME_PLACEHOLDER,
-        MASTER_TABLE_NAME_PLACEHOLDER, null, null, 1);
+        MASTER_TABLE_NAME_PLACEHOLDER, null, null, 1, null);
     this.defaultOperationTimeoutMs = b.defaultOperationTimeoutMs;
     this.defaultAdminOperationTimeoutMs = b.defaultAdminOperationTimeoutMs;
     this.statisticsDisabled = b.statisticsDisabled;
@@ -796,7 +796,8 @@ public class AsyncKuduClient implements AutoCloseable {
             resp.getTableId(),
             resp.getSchema(),
             resp.getPartitionSchema(),
-            resp.getNumReplicas());
+            resp.getNumReplicas(),
+            resp.getExtraConfig());
       }
     });
   }
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java b/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
index 2f137a8..1790037 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
@@ -18,6 +18,7 @@
 package org.apache.kudu.client;
 
 import java.util.List;
+import java.util.Map;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -191,6 +192,19 @@ public class CreateTableOptions {
   }
 
   /**
+   * Sets the table's extra configuration properties.
+   *
+   * If the value of the kv pair is empty, the property will be ignored.
+   *
+   * @param extraConfig the table's extra configuration properties
+   * @return this instance
+   */
+  public CreateTableOptions setExtraConfigs(Map<String, String> extraConfig) {
+    pb.putAllExtraConfigs(extraConfig);
+    return this;
+  }
+
+  /**
    * Whether to wait for the table to be fully created before this create
    * operation is considered to be finished.
    * <p>
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaRequest.java b/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaRequest.java
index 0a7f0f2..0af0cb8 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaRequest.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaRequest.java
@@ -99,7 +99,8 @@ public class GetTableSchemaRequest extends KuduRpc<GetTableSchemaResponse> {
         respBuilder.getTableName(),
         respBuilder.getNumReplicas(),
         ProtobufHelper.pbToPartitionSchema(respBuilder.getPartitionSchema(), schema),
-        respBuilder.hasAuthzToken() ? respBuilder.getAuthzToken() : null);
+        respBuilder.hasAuthzToken() ? respBuilder.getAuthzToken() : null,
+        respBuilder.getExtraConfigsMap());
     return new Pair<GetTableSchemaResponse, Object>(
         response, respBuilder.hasError() ? respBuilder.getError() : null);
   }
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaResponse.java b/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaResponse.java
index a7ff1f1..7e4b557 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaResponse.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/GetTableSchemaResponse.java
@@ -22,6 +22,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.kudu.security.Token.SignedTokenPB;
 import org.apache.kudu.Schema;
 
+import java.util.Map;
+
 @InterfaceAudience.Private
 public class GetTableSchemaResponse extends KuduRpcResponse {
 
@@ -31,6 +33,7 @@ public class GetTableSchemaResponse extends KuduRpcResponse {
   private final String tableName;
   private final int numReplicas;
   private final SignedTokenPB authzToken;
+  private final Map<String, String> extraConfig;
 
   /**
    * @param elapsedMillis Time in milliseconds since RPC creation to now
@@ -41,6 +44,7 @@ public class GetTableSchemaResponse extends KuduRpcResponse {
    * @param numReplicas the table's replication factor
    * @param partitionSchema the table's partition schema
    * @param authzToken an authorization token for use with this table
+   * @param extraConfig the table's extra configuration properties
    */
   GetTableSchemaResponse(long elapsedMillis,
                          String tsUUID,
@@ -49,7 +53,8 @@ public class GetTableSchemaResponse extends KuduRpcResponse {
                          String tableName,
                          int numReplicas,
                          PartitionSchema partitionSchema,
-                         SignedTokenPB authzToken) {
+                         SignedTokenPB authzToken,
+                         Map<String, String> extraConfig) {
     super(elapsedMillis, tsUUID);
     this.schema = schema;
     this.partitionSchema = partitionSchema;
@@ -57,6 +62,7 @@ public class GetTableSchemaResponse extends KuduRpcResponse {
     this.tableName = tableName;
     this.numReplicas = numReplicas;
     this.authzToken = authzToken;
+    this.extraConfig = extraConfig;
   }
 
   /**
@@ -106,4 +112,12 @@ public class GetTableSchemaResponse extends KuduRpcResponse {
   public SignedTokenPB getAuthzToken() {
     return authzToken;
   }
+
+  /**
+   * Get the table's extra configuration properties.
+   * @return the table's extra configuration properties
+   */
+  public Map<String, String> getExtraConfig() {
+    return extraConfig;
+  }
 }
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
index 4c50094..5acb72d 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
@@ -19,6 +19,7 @@ package org.apache.kudu.client;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterators;
@@ -48,6 +49,7 @@ public class KuduTable {
   private final String name;
   private final String tableId;
   private final int numReplicas;
+  private final Map<String, String> extraConfig;
 
   /**
    * Package-private constructor, use {@link KuduClient#openTable(String)} to get an instance.
@@ -57,15 +59,18 @@ public class KuduTable {
    * @param schema this table's schema
    * @param partitionSchema this table's partition schema
    * @param numReplicas this table's replication factor
+   * @param extraConfig this table's extra configuration properties
    */
   KuduTable(AsyncKuduClient client, String name, String tableId,
-            Schema schema, PartitionSchema partitionSchema, int numReplicas) {
+            Schema schema, PartitionSchema partitionSchema, int numReplicas,
+            Map<String, String> extraConfig) {
     this.schema = schema;
     this.partitionSchema = partitionSchema;
     this.client = client;
     this.name = name;
     this.tableId = tableId;
     this.numReplicas = numReplicas;
+    this.extraConfig = extraConfig;
   }
 
   /**
@@ -114,6 +119,14 @@ public class KuduTable {
   }
 
   /**
+   * Get this table's extra configuration properties.
+   * @return this table's extra configuration properties
+   */
+  public Map<String, String> getExtraConfig() {
+    return extraConfig;
+  }
+
+  /**
    * Get the async client that created this instance.
    * @return an async kudu client
    */
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index ac0610f..2f82bc5 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -28,6 +28,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -479,4 +481,44 @@ public class TestAlterTable {
     }
     assertEquals(100, countRowsInTable(table));
   }
+
+  @Test
+  public void testAlterExtraConfigs() throws Exception {
+    KuduTable table = createTable(ImmutableList.<Pair<Integer,Integer>>of());
+    insertRows(table, 0, 100);
+    assertEquals(100, countRowsInTable(table));
+
+    // 1. Check for expected defaults.
+    table = client.openTable(tableName);
+    Map<String, String> extraConfigs = table.getExtraConfig();
+    assertFalse(extraConfigs.containsKey("kudu.table.history_max_age_sec"));
+
+    // 2. Alter history max age second to 3600
+    Map<String, String> alterExtraConfigs = new HashMap<>();
+    alterExtraConfigs.put("kudu.table.history_max_age_sec", "3600");
+    client.alterTable(tableName, new AlterTableOptions().alterExtraConfigs(alterExtraConfigs));
+
+    table = client.openTable(tableName);
+    extraConfigs = table.getExtraConfig();
+    assertTrue(extraConfigs.containsKey("kudu.table.history_max_age_sec"));
+    assertEquals("3600", extraConfigs.get("kudu.table.history_max_age_sec"));
+
+    // 3. Alter history max age second to 7200
+    alterExtraConfigs = new HashMap<>();
+    alterExtraConfigs.put("kudu.table.history_max_age_sec", "7200");
+    client.alterTable(tableName, new AlterTableOptions().alterExtraConfigs(alterExtraConfigs));
+
+    table = client.openTable(tableName);
+    extraConfigs = table.getExtraConfig();
+    assertTrue(extraConfigs.containsKey("kudu.table.history_max_age_sec"));
+    assertEquals("7200", extraConfigs.get("kudu.table.history_max_age_sec"));
+
+    // 4. Reset history max age second to default
+    alterExtraConfigs = new HashMap<>();
+    alterExtraConfigs.put("kudu.table.history_max_age_sec", "");
+    client.alterTable(tableName, new AlterTableOptions().alterExtraConfigs(alterExtraConfigs));
+
+    table = client.openTable(tableName);
+    assertTrue(table.getExtraConfig().isEmpty());
+  }
 }
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
index bef643d..3f5f27d 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
@@ -172,7 +172,7 @@ public class TestAsyncKuduClient {
     // Test that a tablet full of unreachable replicas won't make us retry.
     try {
       KuduTable badTable = new KuduTable(asyncClient, "Invalid table name",
-          "Invalid table ID", null, null, 3);
+          "Invalid table ID", null, null, 3, null);
       asyncClient.discoverTablets(badTable, null, requestBatchSize,
                                   tabletLocations, new ArrayList<>(), 1000);
       fail("This should have failed quickly");
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKeyEncoding.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKeyEncoding.java
index 10215d2..4caf6dd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKeyEncoding.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKeyEncoding.java
@@ -113,7 +113,7 @@ public class TestKeyEncoding {
     Schema schemaOneString =
         buildSchema(new ColumnSchema.ColumnSchemaBuilder("key", Type.STRING).key(true));
     KuduTable table = new KuduTable(null, "one", "one", schemaOneString,
-                                    defaultPartitionSchema(schemaOneString), 3);
+                                    defaultPartitionSchema(schemaOneString), 3, null);
     Insert oneKeyInsert = new Insert(table);
     PartialRow row = oneKeyInsert.getRow();
     row.addString("key", "foo");
@@ -123,7 +123,7 @@ public class TestKeyEncoding {
         new ColumnSchema.ColumnSchemaBuilder("key", Type.STRING).key(true),
         new ColumnSchema.ColumnSchemaBuilder("key2", Type.STRING).key(true));
     KuduTable table2 = new KuduTable(null, "two", "two", schemaTwoString,
-                                     defaultPartitionSchema(schemaTwoString), 3);
+                                     defaultPartitionSchema(schemaTwoString), 3, null);
     Insert twoKeyInsert = new Insert(table2);
     row = twoKeyInsert.getRow();
     row.addString("key", "foo");
@@ -142,7 +142,7 @@ public class TestKeyEncoding {
         new ColumnSchema.ColumnSchemaBuilder("key2", Type.STRING).key(true));
     PartitionSchema partitionSchemaIntString = defaultPartitionSchema(schemaIntString);
     KuduTable table3 = new KuduTable(null, "three", "three",
-        schemaIntString, partitionSchemaIntString, 3);
+        schemaIntString, partitionSchemaIntString, 3, null);
     Insert small = new Insert(table3);
     row = small.getRow();
     row.addInt("key", 20);
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
index dc0aa2f..0f71b5f 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
@@ -47,6 +47,8 @@ import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
@@ -184,6 +186,26 @@ public class TestKuduClient {
     }
   }
 
+  /**
+   * Test creating and deleting a table with extra-configs through a KuduClient.
+   */
+  @Test(timeout = 100000)
+  public void testCreateDeleteTableWitExtraConfigs() throws Exception {
+    // Check that we can create a table.
+    Map<String, String> extraConfigs = new HashMap<>();
+    extraConfigs.put("kudu.table.history_max_age_sec", "7200");
+
+    client.createTable(
+        TABLE_NAME,
+        basicSchema,
+        getBasicCreateTableOptions().setExtraConfigs(extraConfigs));
+
+    KuduTable table = client.openTable(TABLE_NAME);
+    extraConfigs = table.getExtraConfig();
+    assertTrue(extraConfigs.containsKey("kudu.table.history_max_age_sec"));
+    assertEquals("7200", extraConfigs.get("kudu.table.history_max_age_sec"));
+  }
+
   /*
    * Test the scanner behavior when a scanner is used beyond
    * the scanner ttl without calling keepAlive.
diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index 0e2e77e..a303803 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -438,3 +438,11 @@ message KeyRangePB {
   // Number of bytes in chunk.
   required uint64 size_bytes_estimates = 3;
 }
+
+message TableExtraConfigPB {
+  // Number of seconds to retain history for tablets in this table,
+  // including history required to perform diff scans and incremental
+  // backups. Reads initiated at a snapshot that is older than this
+  // age will be rejected. Equivalent to --tablet_history_max_age_sec.
+  optional int32 history_max_age_sec = 1;
+}
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index e7781b7..50d2133 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -23,10 +23,13 @@
 #include <cstring>
 #include <ostream>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
 #include <glog/logging.h>
+#include <google/protobuf/map.h>
+#include <google/protobuf/stubs/common.h>
 
 #include "kudu/common/column_predicate.h"
 #include "kudu/common/columnblock.h"
@@ -40,6 +43,7 @@
 #include "kudu/gutil/fixedarray.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/fastmem.h"
+#include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/bitmap.h"
@@ -53,9 +57,11 @@
 #include "kudu/util/safe_math.h"
 #include "kudu/util/slice.h"
 
+using google::protobuf::Map;
 using google::protobuf::RepeatedPtrField;
 using kudu::pb_util::SecureDebugString;
 using kudu::pb_util::SecureShortDebugString;
+using std::map;
 using std::string;
 using std::vector;
 using strings::Substitute;
@@ -647,6 +653,45 @@ Status ColumnPredicateFromPB(const Schema& schema,
   return Status::OK();
 }
 
+const char kTableHistoryMaxAgeSec[] = "kudu.table.history_max_age_sec";
+Status ExtraConfigPBToMap(const TableExtraConfigPB& pb, map<string, string>* configs) {
+  Map<string, string> tmp;
+  RETURN_NOT_OK(ExtraConfigPBToPBMap(pb, &tmp));
+  map<string, string> result(tmp.begin(), tmp.end());
+  *configs = std::move(result);
+  return Status::OK();
+}
+
+Status ExtraConfigPBFromPBMap(const Map<string, string>& configs, TableExtraConfigPB* pb) {
+  TableExtraConfigPB result;
+  for (const auto& config : configs) {
+    const string& name = config.first;
+    const string& value = config.second;
+    if (name == kTableHistoryMaxAgeSec) {
+      if (!value.empty()) {
+        int32_t history_max_age_sec;
+        if (!safe_strto32(value, &history_max_age_sec)) {
+          return Status::InvalidArgument(Substitute("Unable to parse $0", name), value);
+        }
+        result.set_history_max_age_sec(history_max_age_sec);
+      }
+    } else {
+      LOG(WARNING) << "Unknown extra configuration property: " << name;
+    }
+  }
+  *pb = std::move(result);
+  return Status::OK();
+}
+
+Status ExtraConfigPBToPBMap(const TableExtraConfigPB& pb, Map<string, string>* configs) {
+  Map<string, string> result;
+  if (pb.has_history_max_age_sec()) {
+    result[kTableHistoryMaxAgeSec] = std::to_string(pb.history_max_age_sec());
+  }
+  *configs = std::move(result);
+  return Status::OK();
+}
+
 // Because we use a faststring here, ASAN tests become unbearably slow
 // with the extra verifications.
 ATTRIBUTE_NO_ADDRESS_SAFETY_ANALYSIS
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index 3cb1883..cbc7c60 100644
--- a/src/kudu/common/wire_protocol.h
+++ b/src/kudu/common/wire_protocol.h
@@ -20,6 +20,7 @@
 #define KUDU_COMMON_WIRE_PROTOCOL_H
 
 #include <cstdint>
+#include <map>
 #include <string>
 #include <vector>
 
@@ -33,6 +34,7 @@ class optional;
 namespace google {
 namespace protobuf {
 template <typename Element> class RepeatedPtrField;
+template <typename Key, typename T> class Map;
 }
 }
 
@@ -58,6 +60,7 @@ class RowwiseRowBlockPB;
 class SchemaPB;
 class ServerEntryPB;
 class ServerRegistrationPB;
+class TableExtraConfigPB;
 
 // Convert the given C++ Status object into the equivalent Protobuf.
 void StatusToPB(const Status& status, AppStatusPB* pb);
@@ -139,6 +142,18 @@ Status ColumnPredicateFromPB(const Schema& schema,
                              const ColumnPredicatePB& pb,
                              boost::optional<ColumnPredicate>* predicate);
 
+// Convert a extra configuration properties protobuf to map.
+Status ExtraConfigPBToMap(const TableExtraConfigPB& pb,
+                          std::map<std::string, std::string>* configs);
+
+// Convert the table's extra configuration protobuf::map to protobuf.
+Status ExtraConfigPBFromPBMap(const google::protobuf::Map<std::string, std::string>& configs,
+                              TableExtraConfigPB* pb);
+
+// Convert a extra configuration properties protobuf to protobuf::map.
+Status ExtraConfigPBToPBMap(const TableExtraConfigPB& pb,
+                            google::protobuf::Map<std::string, std::string>* configs);
+
 // Encode the given row block into the provided protobuf and data buffers.
 //
 // All data (both direct and indirect) for each selected row in the RowBlock is
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index eeb351d..8e2ef75 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -66,6 +66,7 @@
 #include <gflags/gflags.h>
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#include <google/protobuf/stubs/common.h>
 
 #include "kudu/cfile/type_encodings.h"
 #include "kudu/common/common.pb.h"
@@ -269,6 +270,7 @@ using base::subtle::NoBarrier_Load;
 using boost::make_optional;
 using boost::none;
 using boost::optional;
+using google::protobuf::Map;
 using kudu::cfile::TypeEncodingInfo;
 using kudu::consensus::ConsensusServiceProxy;
 using kudu::consensus::ConsensusStatePB;
@@ -1581,6 +1583,10 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
         num_live_tservers);
   }
 
+  // Verify the table's extra configuration properties.
+  TableExtraConfigPB extra_config_pb;
+  RETURN_NOT_OK(ExtraConfigPBFromPBMap(req.extra_configs(), &extra_config_pb));
+
   scoped_refptr<TableInfo> table;
   {
     std::lock_guard<LockType> l(lock_);
@@ -1615,7 +1621,7 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
 
   // d. Create the in-memory representation of the new table and its tablets.
   //    It's not yet in any global maps; that will happen in step g below.
-  table = CreateTableInfo(req, schema, partition_schema);
+  table = CreateTableInfo(req, schema, partition_schema, std::move(extra_config_pb));
   vector<scoped_refptr<TabletInfo>> tablets;
   auto abort_mutations = MakeScopedCleanup([&table, &tablets]() {
     table->mutable_metadata()->AbortMutation();
@@ -1744,9 +1750,11 @@ Status CatalogManager::IsCreateTableDone(const IsCreateTableDoneRequestPB* req,
   return Status::OK();
 }
 
-scoped_refptr<TableInfo> CatalogManager::CreateTableInfo(const CreateTableRequestPB& req,
-                                                         const Schema& schema,
-                                                         const PartitionSchema& partition_schema) {
+scoped_refptr<TableInfo> CatalogManager::CreateTableInfo(
+    const CreateTableRequestPB& req,
+    const Schema& schema,
+    const PartitionSchema& partition_schema,
+    TableExtraConfigPB extra_config_pb) {
   DCHECK(schema.has_column_ids());
   scoped_refptr<TableInfo> table = new TableInfo(GenerateId());
   table->mutable_metadata()->StartMutation();
@@ -1761,6 +1769,7 @@ scoped_refptr<TableInfo> CatalogManager::CreateTableInfo(const CreateTableReques
   CHECK_OK(SchemaToPB(schema, metadata->mutable_schema()));
   partition_schema.ToPB(metadata->mutable_partition_schema());
   metadata->set_create_timestamp(time(nullptr));
+  (*metadata->mutable_extra_config()) = std::move(extra_config_pb);
   return table;
 }
 
@@ -2549,10 +2558,25 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
           resp, MasterErrorPB::UNKNOWN_ERROR));
   }
 
+  // 6. Alter table's extra configuration properties.
+  if (!req.new_extra_configs().empty()) {
+    TRACE("Apply alter extra-config");
+    Map<string, string> new_extra_configs;
+    RETURN_NOT_OK(ExtraConfigPBToPBMap(l.data().pb.extra_config(),
+                                       &new_extra_configs));
+    // Merge table's extra configuration properties.
+    for (auto config : req.new_extra_configs()) {
+      new_extra_configs[config.first] = config.second;
+    }
+    RETURN_NOT_OK(ExtraConfigPBFromPBMap(new_extra_configs,
+                                         l.mutable_data()->pb.mutable_extra_config()));
+  }
+
   // Set to true if columns are altered, added or dropped.
   bool has_schema_changes = !alter_schema_steps.empty();
   // Set to true if there are schema changes, or the table is renamed.
-  bool has_metadata_changes = has_schema_changes || req.has_new_table_name();
+  bool has_metadata_changes =
+      has_schema_changes || req.has_new_table_name() || !req.new_extra_configs().empty();
   // Set to true if there are partitioning changes.
   bool has_partitioning_changes = !alter_partitioning_steps.empty();
   // Set to true if metadata changes need to be applied to existing tablets.
@@ -2564,7 +2588,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
     return Status::OK();
   }
 
-  // 6. Serialize the schema and increment the version number.
+  // 7. Serialize the schema and increment the version number.
   if (has_metadata_changes_for_existing_tablets && !l.data().pb.has_fully_applied_schema()) {
     l.mutable_data()->pb.mutable_fully_applied_schema()->CopyFrom(l.data().pb.schema());
   }
@@ -2585,7 +2609,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
                                            LocalTimeAsString()));
   }
 
-  // 7. Update sys-catalog with the new table schema and tablets to add/drop.
+  // 8. Update sys-catalog with the new table schema and tablets to add/drop.
   TRACE("Updating metadata on disk");
   string deletion_msg = "Partition dropped at " + LocalTimeAsString();
   SysCatalogTable::Actions actions;
@@ -2615,7 +2639,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
     return s;
   }
 
-  // 8. Commit the in-memory state.
+  // 9. Commit the in-memory state.
   {
     TRACE("Committing alterations to in-memory state");
 
@@ -2769,6 +2793,7 @@ Status CatalogManager::GetTableSchema(const GetTableSchemaRequestPB* req,
   resp->set_table_id(table->id());
   resp->mutable_partition_schema()->CopyFrom(l.data().pb.partition_schema());
   resp->set_table_name(l.data().pb.name());
+  RETURN_NOT_OK(ExtraConfigPBToPBMap(l.data().pb.extra_config(), resp->mutable_extra_configs()));
 
   return Status::OK();
 }
@@ -3271,6 +3296,8 @@ class AsyncCreateReplica : public RetrySpecificTSRpcTask {
         table_lock.data().pb.partition_schema());
     req_.mutable_config()->CopyFrom(
         tablet_lock.data().pb.consensus_state().committed_config());
+    req_.mutable_extra_config()->CopyFrom(
+        table_lock.data().pb.extra_config());
   }
 
   string type_name() const override { return "Create Tablet"; }
@@ -3469,6 +3496,7 @@ class AsyncAlterTable : public RetryingTSRpcTask {
     req.set_new_table_name(l.data().pb.name());
     req.set_schema_version(l.data().pb.version());
     req.mutable_schema()->CopyFrom(l.data().pb.schema());
+    req.mutable_new_extra_config()->CopyFrom(l.data().pb.extra_config());
 
     l.Unlock();
 
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index e78cbda..282e7c3 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -65,6 +65,7 @@ class PartitionPB;
 class PartitionSchema;
 class Schema;
 class ThreadPool;
+class TableExtraConfigPB;
 struct ColumnId;
 
 // Working around FRIEND_TEST() ugliness.
@@ -854,7 +855,8 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // "dirty" state field.
   scoped_refptr<TableInfo> CreateTableInfo(const CreateTableRequestPB& req,
                                            const Schema& schema,
-                                           const PartitionSchema& partition_schema);
+                                           const PartitionSchema& partition_schema,
+                                           TableExtraConfigPB extra_config_pb);
 
   // Helper for creating the initial TabletInfo state.
   // Leaves the tablet "write locked" with the new info in the
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index 8bbced0..dbf01a7 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -189,6 +189,8 @@ message SysTablesEntryPB {
   optional int64 create_timestamp = 10;
   // The last alter time of the table, in seconds since the epoch.
   optional int64 alter_timestamp = 11;
+  // The table's extra configuration properties.
+  optional TableExtraConfigPB extra_config = 12;
 }
 
 // The on-disk entry in the sys.catalog table ("metadata" column) to represent
@@ -452,6 +454,9 @@ message CreateTableRequestPB {
   // If set, uses the provided value as the table owner when creating the table
   // in external catalogs such as the Hive Metastore.
   optional string owner = 8;
+
+  // The table's extra configuration properties.
+  map<string, string> extra_configs = 9;
 }
 
 message CreateTableResponsePB {
@@ -611,6 +616,8 @@ message AlterTableRequestPB {
   // Whether to apply the alteration to external catalogs, such as the Hive Metastore,
   // which the Kudu master has been configured to integrate with.
   optional bool modify_external_catalogs = 5 [default = true];
+
+  map<string, string> new_extra_configs = 6;
 }
 
 message AlterTableResponsePB {
@@ -671,6 +678,9 @@ message GetTableSchemaResponsePB {
   // A token can always be expected with this response, unless the response is
   // from an older version of Kudu or unless the request resulted in an error.
   optional security.SignedTokenPB authz_token = 8;
+
+  // The table's extra configuration properties.
+  map<string, string> extra_configs = 9;
 }
 
 message ConnectToMasterRequestPB {
diff --git a/src/kudu/master/master_path_handlers.cc b/src/kudu/master/master_path_handlers.cc
index 61abf90..7688796 100644
--- a/src/kudu/master/master_path_handlers.cc
+++ b/src/kudu/master/master_path_handlers.cc
@@ -282,6 +282,7 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
 
   Schema schema;
   PartitionSchema partition_schema;
+  map<string, string> extra_configs;
   vector<scoped_refptr<TabletInfo>> tablets;
   {
     TableMetadataLock l(table.get(), LockMode::READ);
@@ -340,6 +341,12 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
           Substitute("Unable to decode partition schema: $0", s.ToString());
       return;
     }
+    s = ExtraConfigPBToMap(l.data().pb.extra_config(), &extra_configs);
+    if (!s.ok()) {
+      (*output)["error"] =
+          Substitute("Unable to decode extra configuration properties: $0", s.ToString());
+      return;
+    }
     table->GetAllTablets(&tablets);
   }
 
@@ -419,6 +426,10 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
 
   (*output)["partition_schema"] = partition_schema.DisplayString(schema, range_partitions);
 
+  string str_extra_configs;
+  JoinMapKeysAndValues(extra_configs, " : ", "\n", &str_extra_configs);
+  (*output)["extra_config"] = str_extra_configs;
+
   EasyJson summary_json = output->Set("tablets_summary", EasyJson::kArray);
   for (const auto& entry : summary_states) {
     EasyJson state_json = summary_json.PushBack(EasyJson::kObject);
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index a1410ae..e3ed8d9 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -253,6 +253,7 @@ Status SysCatalogTable::CreateNew(FsManager *fs_manager) {
                                                   tablet::TABLET_DATA_READY,
                                                   /*tombstone_last_logged_opid=*/ boost::none,
                                                   /*supports_live_row_count=*/ true,
+                                                  /*extra_config=*/ boost::none,
                                                   &metadata));
 
   RaftConfigPB config;
diff --git a/src/kudu/tablet/metadata.proto b/src/kudu/tablet/metadata.proto
index 57ff067..861c8a3 100644
--- a/src/kudu/tablet/metadata.proto
+++ b/src/kudu/tablet/metadata.proto
@@ -139,6 +139,9 @@ message TabletSuperBlockPB {
   // may be inaccurate and should be ignored. It's only supported for the newly
   // created ones, not for the ancient ones.
   optional bool supports_live_row_count = 16;
+
+  // The table's extra-config.
+  optional TableExtraConfigPB extra_config = 17;
 }
 
 // Tablet states represent stages of a TabletReplica's object lifecycle and are
diff --git a/src/kudu/tablet/tablet-harness.h b/src/kudu/tablet/tablet-harness.h
index c5610ee..b4fd1a7 100644
--- a/src/kudu/tablet/tablet-harness.h
+++ b/src/kudu/tablet/tablet-harness.h
@@ -101,6 +101,7 @@ class TabletHarness {
                                                partition.second,
                                                TABLET_DATA_READY,
                                                /*tombstone_last_logged_opid=*/ boost::none,
+                                               /*extra_config=*/ boost::none,
                                                &metadata));
     if (options_.enable_metrics) {
       metrics_registry_.reset(new MetricRegistry());
diff --git a/src/kudu/tablet/tablet-test-util.h b/src/kudu/tablet/tablet-test-util.h
index 1b67fa9..599f113 100644
--- a/src/kudu/tablet/tablet-test-util.h
+++ b/src/kudu/tablet/tablet-test-util.h
@@ -144,9 +144,13 @@ class KuduTabletTest : public KuduTest {
     return harness_->fs_manager();
   }
 
-  void AlterSchema(const Schema& schema) {
+  void AlterSchema(const Schema& schema,
+                   boost::optional<TableExtraConfigPB> extra_config = boost::none) {
     tserver::AlterSchemaRequestPB req;
     req.set_schema_version(tablet()->metadata()->schema_version() + 1);
+    if (extra_config) {
+      *(req.mutable_new_extra_config()) = *extra_config;
+    }
 
     AlterSchemaTransactionState tx_state(nullptr, &req, nullptr);
     ASSERT_OK(tablet()->CreatePreparedAlterSchema(&tx_state, &schema));
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 9d5909a..1506b6a 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1263,6 +1263,9 @@ Status Tablet::AlterSchema(AlterSchemaTransactionState* tx_state) {
       metric_entity_->SetAttribute("table_name", tx_state->new_table_name());
     }
   }
+  if (tx_state->has_new_extra_config()) {
+    metadata_->SetExtraConfig(tx_state->new_extra_config());
+  }
 
   // If the current schema and the new one are equal, there is nothing to do.
   if (same_schema) {
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index 256043a..0da4e7c 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -127,6 +127,7 @@ class BootstrapTest : public LogTestBase {
                                                partition.second,
                                                TABLET_DATA_READY,
                                                /*tombstone_last_logged_opid=*/ boost::none,
+                                               /*extra_config=*/ boost::none,
                                                meta));
     (*meta)->SetLastDurableMrsIdForTests(mrs_id);
     if ((*meta)->GetRowSetForTests(0) != nullptr) {
diff --git a/src/kudu/tablet/tablet_metadata-test.cc b/src/kudu/tablet/tablet_metadata-test.cc
index 006ee51..dc482eb 100644
--- a/src/kudu/tablet/tablet_metadata-test.cc
+++ b/src/kudu/tablet/tablet_metadata-test.cc
@@ -15,24 +15,28 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/tablet/tablet_metadata.h"
+
 #include <cstdint>
 #include <memory>
 #include <ostream>
 #include <string>
 
+#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
+#include "kudu/common/common.pb.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/common/wire_protocol-test-util.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/metadata.pb.h"
 #include "kudu/tablet/tablet-harness.h"
 #include "kudu/tablet/tablet-test-util.h"
 #include "kudu/tablet/tablet.h"
-#include "kudu/tablet/tablet_metadata.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
@@ -69,38 +73,75 @@ void TestTabletMetadata::BuildPartialRow(int key, int intval, const char* strval
 
 // Test that loading & storing the superblock results in an equivalent file.
 TEST_F(TestTabletMetadata, TestLoadFromSuperBlock) {
+  TabletMetadata* meta = harness_->tablet()->metadata();
+
   // Write some data to the tablet and flush.
   gscoped_ptr<KuduPartialRow> row;
   BuildPartialRow(0, 0, "foo", &row);
-  writer_->Insert(*row);
+  ASSERT_OK(writer_->Insert(*row));
   ASSERT_OK(harness_->tablet()->Flush());
 
   // Create one more rowset. Write and flush.
   BuildPartialRow(1, 1, "bar", &row);
-  writer_->Insert(*row);
+  ASSERT_OK(writer_->Insert(*row));
   ASSERT_OK(harness_->tablet()->Flush());
 
+  // Dump the superblock to a PB. Save the PB to the side.
+  TabletSuperBlockPB superblock_pb_0;
+  ASSERT_OK(meta->ToSuperBlock(&superblock_pb_0));
+
+  // Alter table's extra configuration properties.
+  TableExtraConfigPB extra_config;
+  extra_config.set_history_max_age_sec(7200);
+  NO_FATALS(AlterSchema(*harness_->tablet()->schema(), boost::make_optional(extra_config)));
+
   // Shut down the tablet.
   harness_->tablet()->Shutdown();
 
-  TabletMetadata* meta = harness_->tablet()->metadata();
-
   // Dump the superblock to a PB. Save the PB to the side.
   TabletSuperBlockPB superblock_pb_1;
   ASSERT_OK(meta->ToSuperBlock(&superblock_pb_1));
 
-  // Load the superblock PB back into the TabletMetadata.
-  ASSERT_OK(meta->ReplaceSuperBlock(superblock_pb_1));
-
-  // Dump the tablet metadata to a superblock PB again, and save it.
-  TabletSuperBlockPB superblock_pb_2;
-  ASSERT_OK(meta->ToSuperBlock(&superblock_pb_2));
+  // Check for AlterSchema.
+  ASSERT_FALSE(superblock_pb_0.has_extra_config());
+  ASSERT_TRUE(superblock_pb_1.has_extra_config());
+  ASSERT_TRUE(superblock_pb_1.extra_config().has_history_max_age_sec());
+  ASSERT_EQ(7200, superblock_pb_1.extra_config().history_max_age_sec());
+
+  // Test TabletMetadata::ReplaceSuperBlock
+  {
+    // Load the superblock PB back into the TabletMetadata.
+    ASSERT_OK(meta->ReplaceSuperBlock(superblock_pb_1));
+
+    // Dump the tablet metadata to a superblock PB again, and save it.
+    TabletSuperBlockPB superblock_pb_2;
+    ASSERT_OK(meta->ToSuperBlock(&superblock_pb_2));
+
+    // Compare the 2 dumped superblock PBs.
+    ASSERT_EQ(superblock_pb_1.SerializeAsString(),
+              superblock_pb_2.SerializeAsString())
+                  << pb_util::SecureDebugString(superblock_pb_1)
+                  << pb_util::SecureDebugString(superblock_pb_2);
+  }
 
-  // Compare the 2 dumped superblock PBs.
-  ASSERT_EQ(superblock_pb_1.SerializeAsString(),
-            superblock_pb_2.SerializeAsString())
-    << pb_util::SecureDebugString(superblock_pb_1)
-    << pb_util::SecureDebugString(superblock_pb_2);
+  // Test TabletMetadata::Load
+  {
+    // Load the TabletMetadata from disk.
+    scoped_refptr<TabletMetadata> new_meta;
+    ASSERT_OK(TabletMetadata::Load(harness_->fs_manager(),
+                                   harness_->tablet()->tablet_id(),
+                                   &new_meta));
+
+    // Dump the tablet metadata to a superblock PB again, and save it.
+    TabletSuperBlockPB superblock_pb_2;
+    ASSERT_OK(new_meta->ToSuperBlock(&superblock_pb_2));
+
+    // Compare the 2 dumped superblock PBs.
+    ASSERT_EQ(superblock_pb_1.SerializeAsString(),
+              superblock_pb_2.SerializeAsString())
+                  << pb_util::SecureDebugString(superblock_pb_1)
+                  << pb_util::SecureDebugString(superblock_pb_2);
+  }
 
   LOG(INFO) << "Superblocks match:\n"
             << pb_util::SecureDebugString(superblock_pb_1);
@@ -142,6 +183,5 @@ TEST_F(TestTabletMetadata, TestOnDiskSize) {
   ASSERT_GE(final_size, superblock_pb.ByteSize());
 }
 
-
 } // namespace tablet
 } // namespace kudu
diff --git a/src/kudu/tablet/tablet_metadata.cc b/src/kudu/tablet/tablet_metadata.cc
index ec0d628..18f080c 100644
--- a/src/kudu/tablet/tablet_metadata.cc
+++ b/src/kudu/tablet/tablet_metadata.cc
@@ -26,6 +26,7 @@
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
 
+#include "kudu/common/common.pb.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/opid.pb.h"
@@ -91,6 +92,7 @@ Status TabletMetadata::CreateNew(FsManager* fs_manager,
                                  const TabletDataState& initial_tablet_data_state,
                                  boost::optional<OpId> tombstone_last_logged_opid,
                                  bool supports_live_row_count,
+                                 boost::optional<TableExtraConfigPB> extra_config,
                                  scoped_refptr<TabletMetadata>* metadata) {
 
   // Verify that no existing tablet exists with the same ID.
@@ -112,7 +114,8 @@ Status TabletMetadata::CreateNew(FsManager* fs_manager,
                                                        partition,
                                                        initial_tablet_data_state,
                                                        std::move(tombstone_last_logged_opid),
-                                                       supports_live_row_count));
+                                                       supports_live_row_count,
+                                                       std::move(extra_config)));
   RETURN_NOT_OK(ret->Flush());
   dir_group_cleanup.cancel();
 
@@ -138,6 +141,7 @@ Status TabletMetadata::LoadOrCreate(FsManager* fs_manager,
                                     const Partition& partition,
                                     const TabletDataState& initial_tablet_data_state,
                                     boost::optional<OpId> tombstone_last_logged_opid,
+                                    boost::optional<TableExtraConfigPB> extra_config,
                                     scoped_refptr<TabletMetadata>* metadata) {
   Status s = Load(fs_manager, tablet_id, metadata);
   if (s.ok()) {
@@ -153,6 +157,7 @@ Status TabletMetadata::LoadOrCreate(FsManager* fs_manager,
                      partition_schema, partition, initial_tablet_data_state,
                      std::move(tombstone_last_logged_opid),
                      /*supports_live_row_count=*/ true,
+                     std::move(extra_config),
                      metadata);
   }
   return s;
@@ -271,7 +276,8 @@ TabletMetadata::TabletMetadata(FsManager* fs_manager, string tablet_id,
                                Partition partition,
                                const TabletDataState& tablet_data_state,
                                boost::optional<OpId> tombstone_last_logged_opid,
-                               bool supports_live_row_count)
+                               bool supports_live_row_count,
+                               boost::optional<TableExtraConfigPB> extra_config)
     : state_(kNotWrittenYet),
       tablet_id_(std::move(tablet_id)),
       table_id_(std::move(table_id)),
@@ -285,6 +291,7 @@ TabletMetadata::TabletMetadata(FsManager* fs_manager, string tablet_id,
       partition_schema_(std::move(partition_schema)),
       tablet_data_state_(tablet_data_state),
       tombstone_last_logged_opid_(std::move(tombstone_last_logged_opid)),
+      extra_config_(std::move(extra_config)),
       num_flush_pins_(0),
       needs_flush_(false),
       flush_count_for_tests_(0),
@@ -451,6 +458,12 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
     } else {
       tombstone_last_logged_opid_ = boost::none;
     }
+
+    if (superblock.has_extra_config()) {
+      extra_config_ = superblock.extra_config();
+    } else {
+      extra_config_ = boost::none;
+    }
   }
 
   // Now is a good time to clean up any orphaned blocks that may have been
@@ -689,6 +702,10 @@ Status TabletMetadata::ToSuperBlockUnlocked(TabletSuperBlockPB* super_block,
 
   pb.set_supports_live_row_count(supports_live_row_count_);
 
+  if (extra_config_) {
+    *pb.mutable_extra_config() = *extra_config_;
+  }
+
   super_block->Swap(&pb);
   return Status::OK();
 }
@@ -774,5 +791,15 @@ TabletDataState TabletMetadata::tablet_data_state() const {
   return tablet_data_state_;
 }
 
+void TabletMetadata::SetExtraConfig(TableExtraConfigPB extra_config) {
+  std::lock_guard<LockType> l(data_lock_);
+  extra_config_ = std::move(extra_config);
+}
+
+boost::optional<TableExtraConfigPB> TabletMetadata::extra_config() const {
+  std::lock_guard<LockType> l(data_lock_);
+  return extra_config_;
+}
+
 } // namespace tablet
 } // namespace kudu
diff --git a/src/kudu/tablet/tablet_metadata.h b/src/kudu/tablet/tablet_metadata.h
index 1462950..efb124b 100644
--- a/src/kudu/tablet/tablet_metadata.h
+++ b/src/kudu/tablet/tablet_metadata.h
@@ -44,6 +44,7 @@ namespace kudu {
 class BlockIdPB;
 class FsManager;
 class Schema;
+class TableExtraConfigPB;
 
 namespace consensus {
 class OpId;
@@ -82,6 +83,7 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
                           const TabletDataState& initial_tablet_data_state,
                           boost::optional<consensus::OpId> tombstone_last_logged_opid,
                           bool supports_live_row_count,
+                          boost::optional<TableExtraConfigPB> extra_config,
                           scoped_refptr<TabletMetadata>* metadata);
 
   // Load existing metadata from disk.
@@ -103,6 +105,7 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
                              const Partition& partition,
                              const TabletDataState& initial_tablet_data_state,
                              boost::optional<consensus::OpId> tombstone_last_logged_opid,
+                             boost::optional<TableExtraConfigPB> extra_config,
                              scoped_refptr<TabletMetadata>* metadata);
 
   static std::vector<BlockIdPB> CollectBlockIdPBs(
@@ -133,6 +136,8 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
 
   void SetTableName(const std::string& table_name);
 
+  void SetExtraConfig(TableExtraConfigPB extra_config);
+
   // Return a reference to the current schema.
   // This pointer will be valid until the TabletMetadata is destructed,
   // even if the schema is changed.
@@ -234,6 +239,9 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
   // Return the last-logged opid of a tombstoned tablet, if known.
   boost::optional<consensus::OpId> tombstone_last_logged_opid() const;
 
+  // Returns the table's extra configuration properties.
+  boost::optional<TableExtraConfigPB> extra_config() const;
+
   // Loads the currently-flushed superblock from disk into the given protobuf.
   Status ReadSuperBlockFromDisk(TabletSuperBlockPB* superblock) const;
 
@@ -290,7 +298,8 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
                  Partition partition,
                  const TabletDataState& tablet_data_state,
                  boost::optional<consensus::OpId> tombstone_last_logged_opid,
-                 bool supports_live_row_count);
+                 bool supports_live_row_count,
+                 boost::optional<TableExtraConfigPB> extra_config);
 
   // Constructor for loading an existing tablet.
   TabletMetadata(FsManager* fs_manager, std::string tablet_id);
@@ -378,6 +387,9 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
   // Protected by 'data_lock_'.
   boost::optional<consensus::OpId> tombstone_last_logged_opid_;
 
+  // Table extra config.
+  boost::optional<TableExtraConfigPB> extra_config_;
+
   // If this counter is > 0 then Flush() will not write any data to
   // disk.
   int32_t num_flush_pins_;
diff --git a/src/kudu/tablet/transactions/alter_schema_transaction.h b/src/kudu/tablet/transactions/alter_schema_transaction.h
index b33d1a8..5ef3dd6 100644
--- a/src/kudu/tablet/transactions/alter_schema_transaction.h
+++ b/src/kudu/tablet/transactions/alter_schema_transaction.h
@@ -71,6 +71,14 @@ class AlterSchemaTransactionState : public TransactionState {
     return request_->has_new_table_name();
   }
 
+  TableExtraConfigPB new_extra_config() const {
+    return request_->new_extra_config();
+  }
+
+  bool has_new_extra_config() const {
+    return request_->has_new_extra_config();
+  }
+
   uint32_t schema_version() const {
     return request_->schema_version();
   }
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 3fe573e..a1beeeb 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1678,6 +1678,7 @@ TEST_F(ToolTest, TestLocalReplicaDumpDataDirs) {
       tablet::TABLET_DATA_READY,
       /*tombstone_last_logged_opid=*/ boost::none,
       /*supports_live_row_count=*/ true,
+      /*extra_config=*/ boost::none,
       &meta));
   string stdout;
   NO_FATALS(RunActionStdoutString(Substitute("local_replica dump data_dirs $0 "
@@ -1713,6 +1714,7 @@ TEST_F(ToolTest, TestLocalReplicaDumpMeta) {
                   tablet::TABLET_DATA_READY,
                   /*tombstone_last_logged_opid=*/ boost::none,
                   /*supports_live_row_count=*/ true,
+                  /*extra_config=*/ boost::none,
                   &meta);
   string stdout;
   NO_FATALS(RunActionStdoutString(Substitute("local_replica dump meta $0 "
diff --git a/src/kudu/tserver/mini_tablet_server.cc b/src/kudu/tserver/mini_tablet_server.cc
index f8fc866..56ab111 100644
--- a/src/kudu/tserver/mini_tablet_server.cc
+++ b/src/kudu/tserver/mini_tablet_server.cc
@@ -21,6 +21,7 @@
 #include <utility>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
@@ -147,7 +148,7 @@ Status MiniTabletServer::AddTestTablet(const std::string& table_id,
 
   return server_->tablet_manager()->CreateNewTablet(
       table_id, tablet_id, partition.second, table_id,
-      schema_with_ids, partition.first, config, nullptr);
+      schema_with_ids, partition.first, config, boost::none, nullptr);
 }
 
 vector<string> MiniTabletServer::ListTablets() const {
diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index f4fbe48..daa2218 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -27,6 +27,7 @@
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
+#include "kudu/common/common.pb.h"
 #include "kudu/common/partition.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/wire_protocol.h"
@@ -361,6 +362,7 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
                                             superblock_->tablet_data_state(),
                                             superblock_->tombstone_last_logged_opid(),
                                             remote_superblock_->supports_live_row_count(),
+                                            superblock_->extra_config(),
                                             &meta_));
     TRACE("Wrote new tablet metadata");
 
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index e055dc5..df14533 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -3307,7 +3307,7 @@ TEST_F(TabletServerTest, TestWriteOutOfBounds) {
       "TestWriteOutOfBoundsTable", tabletId,
       partitions[1],
       tabletId, schema, partition_schema,
-      mini_server_->CreateLocalConfig(), nullptr));
+      mini_server_->CreateLocalConfig(), boost::none, nullptr));
 
   ASSERT_OK(WaitForTabletRunning(tabletId));
 
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index a004742..b1b1792 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1019,14 +1019,16 @@ void TabletServiceAdminImpl::CreateTablet(const CreateTabletRequestPB* req,
             << partition_schema.PartitionDebugString(partition, schema);
   VLOG(1) << "Full request: " << SecureDebugString(*req);
 
-  s = server_->tablet_manager()->CreateNewTablet(req->table_id(),
-                                                 req->tablet_id(),
-                                                 partition,
-                                                 req->table_name(),
-                                                 schema,
-                                                 partition_schema,
-                                                 req->config(),
-                                                 nullptr);
+  s = server_->tablet_manager()->CreateNewTablet(
+      req->table_id(),
+      req->tablet_id(),
+      partition,
+      req->table_name(),
+      schema,
+      partition_schema,
+      req->config(),
+      req->has_extra_config() ? boost::make_optional(req->extra_config()) : boost::none,
+      nullptr);
   if (PREDICT_FALSE(!s.ok())) {
     TabletServerErrorPB::Code code;
     if (s.IsAlreadyPresent()) {
diff --git a/src/kudu/tserver/ts_tablet_manager-test.cc b/src/kudu/tserver/ts_tablet_manager-test.cc
index 05a8e75..3cde294 100644
--- a/src/kudu/tserver/ts_tablet_manager-test.cc
+++ b/src/kudu/tserver/ts_tablet_manager-test.cc
@@ -23,6 +23,7 @@
 #include <utility>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
@@ -36,8 +37,9 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/master/master.pb.h"
-#include "kudu/tablet/tablet.h"
 #include "kudu/tablet/tablet-harness.h"
+#include "kudu/tablet/tablet.h"
+#include "kudu/tablet/tablet_metadata.h"
 #include "kudu/tablet/tablet_replica.h"
 #include "kudu/tserver/heartbeater.h"
 #include "kudu/tserver/mini_tablet_server.h"
@@ -71,9 +73,6 @@ using master::TabletReportPB;
 using pb_util::SecureShortDebugString;
 using tablet::TabletReplica;
 
-static const char* const kTabletId = "ffffffffffffffffffffffffffffffff";
-
-
 class TsTabletManagerTest : public KuduTest {
  public:
   TsTabletManagerTest()
@@ -100,6 +99,7 @@ class TsTabletManagerTest : public KuduTest {
 
   Status CreateNewTablet(const std::string& tablet_id,
                          const Schema& schema,
+                         boost::optional<TableExtraConfigPB> extra_config,
                          scoped_refptr<tablet::TabletReplica>* out_tablet_replica) {
     Schema full_schema = SchemaBuilder(schema).Build();
     std::pair<PartitionSchema, Partition> partition = tablet::CreateDefaultPartition(full_schema);
@@ -109,6 +109,7 @@ class TsTabletManagerTest : public KuduTest {
                                                    tablet_id,
                                                    full_schema, partition.first,
                                                    config_,
+                                                   std::move(extra_config),
                                                    &tablet_replica));
     if (out_tablet_replica) {
       (*out_tablet_replica) = tablet_replica;
@@ -147,11 +148,24 @@ class TsTabletManagerTest : public KuduTest {
 };
 
 TEST_F(TsTabletManagerTest, TestCreateTablet) {
+  string tablet1 = "0fffffffffffffffffffffffffffffff";
+  string tablet2 = "1fffffffffffffffffffffffffffffff";
+  scoped_refptr<TabletReplica> replica1;
+  scoped_refptr<TabletReplica> replica2;
+  TableExtraConfigPB extra_config;
+  extra_config.set_history_max_age_sec(7200);
+
   // Create a new tablet.
-  scoped_refptr<TabletReplica> replica;
-  ASSERT_OK(CreateNewTablet(kTabletId, schema_, &replica));
-  ASSERT_EQ(kTabletId, replica->tablet()->tablet_id());
-  replica.reset();
+  ASSERT_OK(CreateNewTablet(tablet1, schema_, boost::none, &replica1));
+  // Create a new tablet with extra config.
+  ASSERT_OK(CreateNewTablet(tablet2, schema_, extra_config, &replica2));
+  ASSERT_EQ(tablet1, replica1->tablet()->tablet_id());
+  ASSERT_EQ(tablet2, replica2->tablet()->tablet_id());
+  ASSERT_EQ(boost::none, replica1->tablet()->metadata()->extra_config());
+  ASSERT_NE(boost::none, replica2->tablet()->metadata()->extra_config());
+  ASSERT_EQ(7200, replica2->tablet()->metadata()->extra_config()->history_max_age_sec());
+  replica1.reset();
+  replica2.reset();
 
   // Re-load the tablet manager from the filesystem.
   LOG(INFO) << "Shutting down tablet manager";
@@ -164,8 +178,13 @@ TEST_F(TsTabletManagerTest, TestCreateTablet) {
   tablet_manager_ = mini_server_->server()->tablet_manager();
 
   // Ensure that the tablet got re-loaded and re-opened off disk.
-  ASSERT_TRUE(tablet_manager_->LookupTablet(kTabletId, &replica));
-  ASSERT_EQ(kTabletId, replica->tablet()->tablet_id());
+  ASSERT_TRUE(tablet_manager_->LookupTablet(tablet1, &replica1));
+  ASSERT_TRUE(tablet_manager_->LookupTablet(tablet2, &replica2));
+  ASSERT_EQ(tablet1, replica1->tablet()->tablet_id());
+  ASSERT_EQ(tablet2, replica2->tablet()->tablet_id());
+  ASSERT_EQ(boost::none, replica1->tablet()->metadata()->extra_config());
+  ASSERT_NE(boost::none, replica2->tablet()->metadata()->extra_config());
+  ASSERT_EQ(7200, replica2->tablet()->metadata()->extra_config()->history_max_age_sec());
 }
 
 static void AssertMonotonicReportSeqno(int64_t* report_seqno,
@@ -219,7 +238,7 @@ TEST_F(TsTabletManagerTest, TestTabletReports) {
   MarkTabletReportAcknowledged(report);
 
   // Create a tablet and do another incremental report - should include the tablet.
-  ASSERT_OK(CreateNewTablet("tablet-1", schema_, nullptr));
+  ASSERT_OK(CreateNewTablet("tablet-1", schema_, boost::none, nullptr));
   int updated_tablets = 0;
   while (updated_tablets != 1) {
     GenerateIncrementalTabletReport(&report);
@@ -247,7 +266,7 @@ TEST_F(TsTabletManagerTest, TestTabletReports) {
   MarkTabletReportAcknowledged(report);
 
   // Create a second tablet, and ensure the incremental report shows it.
-  ASSERT_OK(CreateNewTablet("tablet-2", schema_, nullptr));
+  ASSERT_OK(CreateNewTablet("tablet-2", schema_, boost::none, nullptr));
 
   // Wait up to 10 seconds to get a tablet report from tablet-2.
   // TabletReplica does not mark tablets dirty until after it commits the
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 9531f09..9366c68 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -33,6 +33,7 @@
 #include <glog/logging.h>
 
 #include "kudu/clock/clock.h"
+#include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/consensus.pb.h"
@@ -405,6 +406,7 @@ Status TSTabletManager::CreateNewTablet(const string& table_id,
                                         const Schema& schema,
                                         const PartitionSchema& partition_schema,
                                         RaftConfigPB config,
+                                        boost::optional<TableExtraConfigPB> extra_config,
                                         scoped_refptr<TabletReplica>* replica) {
   CHECK_EQ(state(), MANAGER_RUNNING);
   CHECK(IsRaftConfigMember(server_->instance_pb().permanent_uuid(), config));
@@ -443,6 +445,7 @@ Status TSTabletManager::CreateNewTablet(const string& table_id,
                               TABLET_DATA_READY,
                               boost::none,
                               /*supports_live_row_count=*/ true,
+                              std::move(extra_config),
                               &meta),
     "Couldn't create tablet metadata");
 
diff --git a/src/kudu/tserver/ts_tablet_manager.h b/src/kudu/tserver/ts_tablet_manager.h
index 30e1b9c..6ed049a 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -48,6 +48,10 @@ class optional;
 }
 
 namespace kudu {
+class TableExtraConfigPB;
+}  // namespace kudu
+
+namespace kudu {
 
 class FsManager;
 class NodeInstancePB;
@@ -120,6 +124,7 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
                          const Schema& schema,
                          const PartitionSchema& partition_schema,
                          consensus::RaftConfigPB config,
+                         boost::optional<TableExtraConfigPB> extra_config,
                          scoped_refptr<tablet::TabletReplica>* replica);
 
   // Delete the specified tablet asynchronously with callback 'cb'.
diff --git a/src/kudu/tserver/tserver_admin.proto b/src/kudu/tserver/tserver_admin.proto
index 24937ff..4d77441 100644
--- a/src/kudu/tserver/tserver_admin.proto
+++ b/src/kudu/tserver/tserver_admin.proto
@@ -37,6 +37,7 @@ message AlterSchemaRequestPB {
 
   required uint32 schema_version = 3;
   optional string new_table_name = 4;
+  optional TableExtraConfigPB new_extra_config = 6;
 }
 
 message AlterSchemaResponsePB {
@@ -68,6 +69,9 @@ message CreateTabletRequestPB {
 
   // Initial consensus configuration for the tablet.
   required consensus.RaftConfigPB config = 7;
+
+  // The table's extra-config.
+  optional TableExtraConfigPB extra_config = 11;
 }
 
 message CreateTabletResponsePB {
diff --git a/www/table.mustache b/www/table.mustache
index fc82a80..cd12504 100644
--- a/www/table.mustache
+++ b/www/table.mustache
@@ -68,6 +68,9 @@ under the License.
   <h3>Partition Schema</h3>
   <pre>{{partition_schema}}</pre>
 
+  <h3>Extra Config</h3>
+  <pre>{{extra_config}}</pre>
+
   <h3>Tablets</h3>
   <h4>Summary</h4>