You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/01/15 00:33:12 UTC

[kudu] 02/03: [master] KUDU-3016 flag for chunking tablet report updates

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

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

commit 3a30cc22684512e3f2934cde57bc10bebf7dd156
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Jan 13 17:58:07 2020 -0800

    [master] KUDU-3016 flag for chunking tablet report updates
    
    This patch introduces a flag to chunk updates on the system tablet
    generated by master while processing tablet reports.  When the flag
    is set to 'true' (that's the default setting), masters chunk the updates
    on the system tablets which otherwise would be oversized.  When the flag
    is set to 'false', masters reject tablet reports which would lead
    to the oversized write requests on the system tablet.  With either
    setting, masters avoid hitting the maximum RPC size limit while pushing
    corresponding Raft updates on the system tablet to follower masters.
    
    A test is added to reproduce the scenario described in KUDU-3016.
    In the test scenario, the average size of incoming TSHeartbeat RPC
    is about 28 KB in size, while the corresponding WriteRequestPB is
    over 70 KB in size.
    
    Change-Id: I83e8ca4bc8db7cab8fee6b4a40f48adc8752e7c5
    Reviewed-on: http://gerrit.cloudera.org:8080/14897
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/common/row_operations.cc                  |  85 ++++++---
 src/kudu/common/row_operations.h                   |  23 ++-
 .../integration-tests/master_replication-itest.cc  | 100 +++++++++-
 src/kudu/master/catalog_manager.cc                 | 204 +++++++++++++--------
 src/kudu/master/sys_catalog-test.cc                |  70 ++++---
 src/kudu/master/sys_catalog.cc                     | 122 +++++++++---
 src/kudu/master/sys_catalog.h                      |  66 ++++++-
 7 files changed, 501 insertions(+), 169 deletions(-)

diff --git a/src/kudu/common/row_operations.cc b/src/kudu/common/row_operations.cc
index 0603b3b..548209f 100644
--- a/src/kudu/common/row_operations.cc
+++ b/src/kudu/common/row_operations.cc
@@ -96,34 +96,37 @@ void DecodedRowOperation::SetFailureStatusOnce(Status s) {
 }
 
 RowOperationsPBEncoder::RowOperationsPBEncoder(RowOperationsPB* pb)
-  : pb_(pb) {
+    : pb_(pb),
+      prev_indirect_data_size_(string::npos),
+      prev_rows_size_(string::npos) {
 }
 
 RowOperationsPBEncoder::~RowOperationsPBEncoder() {
 }
 
-void RowOperationsPBEncoder::Add(RowOperationsPB::Type op_type, const KuduPartialRow& partial_row) {
+size_t RowOperationsPBEncoder::Add(RowOperationsPB::Type op_type,
+                                   const KuduPartialRow& partial_row) {
   const Schema* schema = partial_row.schema();
 
-  // See wire_protocol.pb for a description of the format.
+  // See wire_protocol.proto for a description of the format.
   string* dst = pb_->mutable_rows();
+  prev_rows_size_ = dst->size();
+  prev_indirect_data_size_ = pb_->mutable_indirect_data()->size();
 
-  // Compute a bound on much space we may need in the 'rows' field.
+  // Compute a bound on how much space we may need in the 'rows' field.
   // Then, resize it to this much space. This allows us to use simple
   // memcpy() calls to copy the data, rather than string->append(), which
   // reduces branches significantly in this fairly hot code path.
   // (std::string::append doesn't get inlined).
   // At the end of the function, we'll resize() the string back down to the
   // right size.
-  int isset_bitmap_size = BitmapSize(schema->num_columns());
-  int null_bitmap_size = ContiguousRowHelper::null_bitmap_size(*schema);
-  int type_size = 1; // type uses one byte
-  int max_size = type_size + schema->byte_size() + isset_bitmap_size + null_bitmap_size;
-  int old_size = dst->size();
-  dst->resize(dst->size() + max_size);
-
-  uint8_t* dst_ptr = reinterpret_cast<uint8_t*>(&(*dst)[old_size]);
+  size_t isset_bitmap_size;
+  size_t null_bitmap_size;
+  const size_t new_size_estimate = GetRowsFieldSizeEstimate(
+      partial_row, &isset_bitmap_size, &null_bitmap_size);
+  dst->resize(new_size_estimate);
 
+  uint8_t* dst_ptr = reinterpret_cast<uint8_t*>(&(*dst)[prev_rows_size_]);
   *dst_ptr++ = static_cast<uint8_t>(op_type);
   memcpy(dst_ptr, partial_row.isset_bitmap_, isset_bitmap_size);
   dst_ptr += isset_bitmap_size;
@@ -133,20 +136,26 @@ void RowOperationsPBEncoder::Add(RowOperationsPB::Type op_type, const KuduPartia
          null_bitmap_size);
   dst_ptr += null_bitmap_size;
 
+  size_t indirect_data_size_delta = 0;
   ContiguousRow row(schema, partial_row.row_data_);
-  for (int i = 0; i < schema->num_columns(); i++) {
-    if (!partial_row.IsColumnSet(i)) continue;
-    const ColumnSchema& col = schema->column(i);
+  for (auto i = 0; i < schema->num_columns(); ++i) {
+    if (!partial_row.IsColumnSet(i)) {
+      continue;
+    }
 
-    if (col.is_nullable() && row.is_null(i)) continue;
+    const ColumnSchema& col = schema->column(i);
+    if (col.is_nullable() && row.is_null(i)) {
+      continue;
+    }
 
     if (col.type_info()->physical_type() == BINARY) {
+      const size_t indirect_offset = pb_->mutable_indirect_data()->size();
       const Slice* val = reinterpret_cast<const Slice*>(row.cell_ptr(i));
-      size_t indirect_offset = pb_->mutable_indirect_data()->size();
-      pb_->mutable_indirect_data()->append(reinterpret_cast<const char*>(val->data()),
-                                           val->size());
-      Slice to_append(reinterpret_cast<const uint8_t*>(indirect_offset),
-                      val->size());
+      indirect_data_size_delta += val->size();
+      pb_->mutable_indirect_data()->append(
+          reinterpret_cast<const char*>(val->data()), val->size());
+      Slice to_append(
+          reinterpret_cast<const uint8_t*>(indirect_offset), val->size());
       memcpy(dst_ptr, &to_append, sizeof(Slice));
       dst_ptr += sizeof(Slice);
     } else {
@@ -155,7 +164,39 @@ void RowOperationsPBEncoder::Add(RowOperationsPB::Type op_type, const KuduPartia
     }
   }
 
-  dst->resize(reinterpret_cast<char*>(dst_ptr) - &(*dst)[0]);
+  const size_t rows_size_delta = reinterpret_cast<uintptr_t>(dst_ptr) -
+      reinterpret_cast<uintptr_t>(&(*dst)[prev_rows_size_]);
+  dst->resize(prev_rows_size_ + rows_size_delta);
+
+  return rows_size_delta + indirect_data_size_delta;
+}
+
+void RowOperationsPBEncoder::RemoveLast() {
+  CHECK_NE(string::npos, prev_indirect_data_size_);
+  CHECK_NE(string::npos, prev_rows_size_);
+  pb_->mutable_indirect_data()->resize(prev_indirect_data_size_);
+  pb_->mutable_rows()->resize(prev_rows_size_);
+  prev_indirect_data_size_ = string::npos;
+  prev_rows_size_ = string::npos;
+}
+
+size_t RowOperationsPBEncoder::GetRowsFieldSizeEstimate(
+    const KuduPartialRow& partial_row,
+    size_t* isset_bitmap_size,
+    size_t* isnull_bitmap_size) const {
+  const Schema* schema = partial_row.schema();
+
+  // See wire_protocol.proto for a description of the format.
+  const string* dst = pb_->mutable_rows();
+
+  auto isset_size = BitmapSize(schema->num_columns());
+  auto isnull_size = ContiguousRowHelper::null_bitmap_size(*schema);
+  constexpr auto type_size = 1; // type uses one byte
+  auto max_size = type_size + schema->byte_size() + isset_size + isnull_size;
+
+  *isset_bitmap_size = isset_size;
+  *isnull_bitmap_size = isnull_size;
+  return dst->size() + max_size;
 }
 
 // ------------------------------------------------------------
diff --git a/src/kudu/common/row_operations.h b/src/kudu/common/row_operations.h
index 0f5224e..e3e441a 100644
--- a/src/kudu/common/row_operations.h
+++ b/src/kudu/common/row_operations.h
@@ -44,11 +44,30 @@ class RowOperationsPBEncoder {
   explicit RowOperationsPBEncoder(RowOperationsPB* pb);
   ~RowOperationsPBEncoder();
 
-  // Append this partial row to the protobuf.
-  void Add(RowOperationsPB::Type type, const KuduPartialRow& row);
+  // Whether there is no row operations in the encoded protobuf message.
+  bool empty() const {
+    return pb_->mutable_rows()->empty();
+  }
+
+  // Append this partial row to the protobuf. Returns the size delta for the
+  // underlying protobuf after adding the partial row.
+  size_t Add(RowOperationsPB::Type type, const KuduPartialRow& partial_row);
+
+  // Remove the last added row from the underlying protobuf. Calling this method
+  // more than one time in a row or when no rows were added triggers
+  // CHECK()/abort.
+  void RemoveLast();
 
  private:
+  // Get the size estimation (upper boundary) for encoded RowOperationsPB::rows
+  // after adding the extra row specified.
+  size_t GetRowsFieldSizeEstimate(const KuduPartialRow& partial_row,
+                                  size_t* isset_bitmap_size,
+                                  size_t* isnull_bitmap_size) const;
+
   RowOperationsPB* pb_;
+  std::string::size_type prev_indirect_data_size_;
+  std::string::size_type prev_rows_size_;
 
   DISALLOW_COPY_AND_ASSIGN(RowOperationsPBEncoder);
 };
diff --git a/src/kudu/integration-tests/master_replication-itest.cc b/src/kudu/integration-tests/master_replication-itest.cc
index a36f907..1493f5a 100644
--- a/src/kudu/integration-tests/master_replication-itest.cc
+++ b/src/kudu/integration-tests/master_replication-itest.cc
@@ -38,6 +38,8 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/integration-tests/cluster_itest_util.h"
+#include "kudu/integration-tests/cluster_verifier.h"
+#include "kudu/integration-tests/test_workload.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
@@ -374,13 +376,8 @@ class MasterReplicationAndRpcSizeLimitTest : public KuduTest {
   static constexpr auto kNumMasters = 3;
   static constexpr auto kNumTabletServers = 3;
   static constexpr auto kReplicationFactor = 3;
-  // In case of standard builds, shorten the Raft hearbeat and election timeout
-  // intervals to speed up the test.
-#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER)
+  // Shorten the Raft election timeout intervals to speed up the test.
   static constexpr auto kHbIntervalMs = 500;
-#else
-  static constexpr auto kHbIntervalMs = 200;
-#endif
   static constexpr auto kMaxMissedHbs = 2;
 
   Status Prepare() {
@@ -408,6 +405,10 @@ class MasterReplicationAndRpcSizeLimitTest : public KuduTest {
       // Substracting an extra 1K to account for extra fields while wrapping
       // messages to replicate into UpdateConsensus RPC.
       Substitute("--consensus_max_batch_size_bytes=$0", 63 * 1024),
+      // The TabletReports scenario first verifies that master rejects tablet
+      // reports which would lead to oversized updates on the system catalog
+      // tablet, and then it toggles the flag in run time.
+      "--catalog_manager_enable_chunked_tablet_reports=false",
     };
 
     ExternalMiniClusterOptions opts;
@@ -431,6 +432,7 @@ class MasterReplicationAndRpcSizeLimitTest : public KuduTest {
         "_very_looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
         "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
         "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
+        "oooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooo"
         "ooooooooooooooooooooooooooooooooooooooooooooooooonooooog_label_suffix";
     KuduSchemaBuilder b;
     b.AddColumn("key")->Type(KuduColumnSchema::INT64)->NotNull()->PrimaryKey();
@@ -468,6 +470,7 @@ class MasterReplicationAndRpcSizeLimitTest : public KuduTest {
 // Make sure leader master rejects AlterTable requests which result in updates
 // on the system tablet which it would not be able to push to its followers
 // due to the limit set by the --rpc_max_message_size flag.
+//
 // This scenario simulates conditions described in KUDU-3036.
 TEST_F(MasterReplicationAndRpcSizeLimitTest, AlterTable) {
   const string table_name = "table_to_alter";
@@ -502,5 +505,90 @@ TEST_F(MasterReplicationAndRpcSizeLimitTest, AlterTable) {
   NO_FATALS(cluster_->AssertNoCrashes());
 }
 
+// In this scenario, Kudu tablet servers send Kudu master tablet reports which
+// are under the maximum RPC size limit, however the corresponding update
+// on the system tablet would be greater than that if lumping together updates
+// for every tablet. If the --catalog_manager_enable_chunked_tablet_reports
+// flag is set to 'false', Kudu masters should reject such reports. If the
+// flag set to 'true', Kudu masters should chunk the result write request to
+// the system catalog, so corresponding UpdateConsensus RPCs are not rejected
+// by follower masters due to the limit on the maximum RPC size.
+//
+// This scenario simulates conditions described in KUDU-3016.
+TEST_F(MasterReplicationAndRpcSizeLimitTest, TabletReports) {
+  SKIP_IF_SLOW_NOT_ALLOWED();
+
+  for (auto idx = 0; idx < 10; ++idx) {
+    ASSERT_OK(CreateTable(Substitute("table_$0", idx), kReplicationFactor));
+  }
+
+  // After fresh start, there should be no rejected writes to the system catalog
+  // tablet yet.
+  int64_t val;
+  ASSERT_OK(GetMetricValue(METRIC_sys_catalog_oversized_write_requests, &val));
+  ASSERT_EQ(0, val);
+
+  // Stop all masters: they will be restarted later to receive tablet reports.
+  for (auto idx = 0; idx < kNumMasters; ++idx) {
+    cluster_->master(idx)->Shutdown();
+  }
+
+  // Pause and resume tablet servers to make the tablets re-elect their leaders,
+  // so Raft configuration for every tablet is updated in the end of the cycle
+  // because of the fresh Raft terms. The result distribution of leader replicas
+  // makes them concentrated at two tablet servers out of three, which results
+  // in two larger tablet reports.
+  for (auto idx = 0; idx < kNumTabletServers; ++idx) {
+    ASSERT_OK(cluster_->tablet_server(idx)->Pause());
+    // Allow for leader re-election to happen.
+    SleepFor(MonoDelta::FromMilliseconds(3 * kMaxMissedHbs * kHbIntervalMs));
+    ASSERT_OK(cluster_->tablet_server(idx)->Resume());
+    SleepFor(MonoDelta::FromMilliseconds(2 * kHbIntervalMs));
+  }
+
+  // Start all masters. The tablet servers should send full (non-incremental)
+  // tablet reports to the leader master once hearing from it.
+  for (auto idx = 0; idx < kNumMasters; ++idx) {
+    ASSERT_OK(cluster_->master(idx)->Restart());
+  }
+
+  // Since the chunked updates on the system catalog tablet is disabled by
+  // default, masters should reject tablet reports that would result in
+  // oversized updates on the system catalog tablet.
+  ASSERT_EVENTUALLY([&] {
+    int64_t val;
+    ASSERT_OK(GetMetricValue(METRIC_sys_catalog_oversized_write_requests, &val));
+    ASSERT_GT(val, 0);
+  });
+
+  for (auto idx = 0; idx < kNumMasters; ++idx) {
+    ASSERT_OK(cluster_->SetFlag(cluster_->master(idx),
+                                "catalog_manager_enable_chunked_tablet_reports",
+                                "true"));
+  }
+
+  ASSERT_OK(cluster_->WaitForTabletServerCount(
+      kNumTabletServers, MonoDelta::FromSeconds(60)));
+
+  // Run a test workload and make sure the system is operable. Prior to
+  // KUDU-3016 fix, the scenario above would lead to a DoS situation.
+  TestWorkload w(cluster_.get());
+  w.set_write_pattern(TestWorkload::INSERT_SEQUENTIAL_ROWS);
+  w.set_num_replicas(kReplicationFactor);
+  w.set_num_write_threads(2);
+  w.set_num_read_threads(2);
+  w.Setup();
+  w.Start();
+  SleepFor(MonoDelta::FromSeconds(3));
+  w.StopAndJoin();
+
+  NO_FATALS(cluster_->AssertNoCrashes());
+
+  ClusterVerifier v(cluster_.get());
+  NO_FATALS(v.CheckCluster());
+  NO_FATALS(v.CheckRowCount(
+      w.table_name(), ClusterVerifier::EXACTLY, w.rows_inserted()));
+}
+
 } // namespace master
 } // namespace kudu
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 027ef6c..a2bc94d 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -279,6 +279,19 @@ DEFINE_bool(catalog_manager_support_live_row_count, true,
 TAG_FLAG(catalog_manager_support_live_row_count, hidden);
 TAG_FLAG(catalog_manager_support_live_row_count, runtime);
 
+DEFINE_bool(catalog_manager_enable_chunked_tablet_reports, true,
+            "Whether to split the tablet report data received from one tablet "
+            "server into chunks when persisting it in the system catalog. "
+            "The chunking starts at around the maximum allowed RPC size "
+            "controlled by the --rpc_max_message_size flag. When the chunking "
+            "is disabled, a tablet report sent by a tablet server is rejected "
+            "if it would result in an oversized update on the system catalog "
+            "tablet. With the default settings for --rpc_max_message_size, "
+            "the latter can happen only in case of extremely high number "
+            "of tablet replicas per tablet server.");
+TAG_FLAG(catalog_manager_enable_chunked_tablet_reports, advanced);
+TAG_FLAG(catalog_manager_enable_chunked_tablet_reports, runtime);
+
 DEFINE_int64(on_disk_size_for_testing, 0,
              "Mock the on disk size of metrics for testing.");
 TAG_FLAG(on_disk_size_for_testing, hidden);
@@ -1717,15 +1730,17 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   });
 
   // f. Write table and tablets to sys-catalog.
-  SysCatalogTable::Actions actions;
-  actions.table_to_add = table;
-  actions.tablets_to_add = tablets;
-  Status s = sys_catalog_->Write(actions);
-  if (!s.ok()) {
-    s = s.CloneAndPrepend("an error occurred while writing to the sys-catalog");
-    LOG(WARNING) << s.ToString();
-    CheckIfNoLongerLeaderAndSetupError(s, resp);
-    return s;
+  {
+    SysCatalogTable::Actions actions;
+    actions.table_to_add = table;
+    actions.tablets_to_add = tablets;
+    Status s = sys_catalog_->Write(std::move(actions));
+    if (PREDICT_FALSE(!s.ok())) {
+      s = s.CloneAndPrepend("an error occurred while writing to the sys-catalog");
+      LOG(WARNING) << s.ToString();
+      CheckIfNoLongerLeaderAndSetupError(s, resp);
+      return s;
+    }
   }
   TRACE("Wrote table and tablets to system table");
 
@@ -2065,16 +2080,19 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB& req,
 
     // 3. Update sys-catalog with the removed table and tablet state.
     TRACE("Removing table and tablets from system table");
-    SysCatalogTable::Actions actions;
-    actions.hms_notification_log_event_id = hms_notification_log_event_id;
-    actions.table_to_update = table;
-    actions.tablets_to_update.assign(tablets.begin(), tablets.end());
-    Status s = sys_catalog_->Write(actions);
-    if (!s.ok()) {
-      s = s.CloneAndPrepend("an error occurred while updating the sys-catalog");
-      LOG(WARNING) << s.ToString();
-      CheckIfNoLongerLeaderAndSetupError(s, resp);
-      return s;
+    {
+      SysCatalogTable::Actions actions;
+      actions.hms_notification_log_event_id =
+          std::move(hms_notification_log_event_id);
+      actions.table_to_update = table;
+      actions.tablets_to_update.assign(tablets.begin(), tablets.end());
+      Status s = sys_catalog_->Write(std::move(actions));
+      if (PREDICT_FALSE(!s.ok())) {
+        s = s.CloneAndPrepend("an error occurred while updating the sys-catalog");
+        LOG(WARNING) << s.ToString();
+        CheckIfNoLongerLeaderAndSetupError(s, resp);
+        return s;
+      }
     }
 
     // 4. Remove the table from the by-name map.
@@ -2664,40 +2682,43 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
                                            LocalTimeAsString()));
   }
 
+  const string deletion_msg = "Partition dropped at " + LocalTimeAsString();
+  TabletMetadataGroupLock tablets_to_add_lock(LockMode::WRITE);
+  TabletMetadataGroupLock tablets_to_drop_lock(LockMode::RELEASED);
+
   // 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;
-  actions.hms_notification_log_event_id = hms_notification_log_event_id;
-  if (!tablets_to_add.empty() || has_metadata_changes) {
-    // If anything modified the table's persistent metadata, then sync it to the sys catalog.
-    actions.table_to_update = table;
-  }
-  actions.tablets_to_add = tablets_to_add;
+  {
+    SysCatalogTable::Actions actions;
+    actions.hms_notification_log_event_id =
+        std::move(hms_notification_log_event_id);
+    if (!tablets_to_add.empty() || has_metadata_changes) {
+      // If anything modified the table's persistent metadata, then sync it to the sys catalog.
+      actions.table_to_update = table;
+    }
+    actions.tablets_to_add = tablets_to_add;
 
-  TabletMetadataGroupLock tablets_to_add_lock(LockMode::WRITE);
-  TabletMetadataGroupLock tablets_to_drop_lock(LockMode::RELEASED);
-  tablets_to_add_lock.AddMutableInfos(tablets_to_add);
-  tablets_to_drop_lock.AddMutableInfos(tablets_to_drop);
-  tablets_to_drop_lock.Lock(LockMode::WRITE);
-  for (auto& tablet : tablets_to_drop) {
-    tablet->mutable_metadata()->mutable_dirty()->set_state(SysTabletsEntryPB::DELETED,
-                                                           deletion_msg);
-  }
-  actions.tablets_to_update = tablets_to_drop;
+    tablets_to_add_lock.AddMutableInfos(tablets_to_add);
+    tablets_to_drop_lock.AddMutableInfos(tablets_to_drop);
+    tablets_to_drop_lock.Lock(LockMode::WRITE);
+    for (auto& tablet : tablets_to_drop) {
+      tablet->mutable_metadata()->mutable_dirty()->set_state(
+          SysTabletsEntryPB::DELETED, deletion_msg);
+    }
+    actions.tablets_to_update = tablets_to_drop;
 
-  Status s = sys_catalog_->Write(actions);
-  if (!s.ok()) {
-    s = s.CloneAndPrepend("an error occurred while updating the sys-catalog");
-    LOG(WARNING) << s.ToString();
-    CheckIfNoLongerLeaderAndSetupError(s, resp);
-    return s;
+    Status s = sys_catalog_->Write(std::move(actions));
+    if (PREDICT_FALSE(!s.ok())) {
+      s = s.CloneAndPrepend("an error occurred while updating the sys-catalog");
+      LOG(WARNING) << s.ToString();
+      CheckIfNoLongerLeaderAndSetupError(s, resp);
+      return s;
+    }
   }
 
   // 9. Commit the in-memory state.
+  TRACE("Committing alterations to in-memory state");
   {
-    TRACE("Committing alterations to in-memory state");
-
     // Commit new tablet in-memory state. This doesn't require taking the global
     // lock since the new tablets are not yet visible, because they haven't been
     // added to the table or tablet index.
@@ -4268,15 +4289,30 @@ Status CatalogManager::ProcessTabletReport(
   // 12. Write all tablet mutations to the catalog table.
   //
   // SysCatalogTable::Write will short-circuit the case where the data has not
-  // in fact changed since the previous version and avoid any unnecessary mutations.
-  SysCatalogTable::Actions actions;
-  actions.tablets_to_update = std::move(mutated_tablets);
-  Status s = sys_catalog_->Write(actions);
-  if (!s.ok()) {
-    LOG(ERROR) << Substitute(
-        "Error updating tablets from $0: $1. Tablet report was: $2",
-        ts_desc->permanent_uuid(), s.ToString(), SecureShortDebugString(full_report));
-    return s;
+  // in fact changed since the previous version and avoid any unnecessary
+  // mutations. The generated sequence of actions may be split into multiple
+  // writes to the system catalog tablet to keep the size of each write request
+  // under the specified threshold.
+  {
+    SysCatalogTable::Actions actions;
+    actions.tablets_to_update = std::move(mutated_tablets);
+    // Updating the status of replicas on the same tablet server can be safely
+    // chunked. Even if some chunks of the update fails, it should not lead to
+    // bigger inconsistencies than simply not updating the status of a single
+    // replica on that tablet server (i.e., rejecting the whole tablet report).
+    // In addition, the nature of such failures is transient, and it's expected
+    // that the next successfully processed tablet report from the tablet server
+    // will fix the partial update.
+    const auto write_mode = FLAGS_catalog_manager_enable_chunked_tablet_reports
+        ? SysCatalogTable::WriteMode::CHUNKED
+        : SysCatalogTable::WriteMode::ATOMIC;
+    auto s = sys_catalog_->Write(std::move(actions), write_mode);
+    if (PREDICT_FALSE(!s.ok())) {
+      LOG(ERROR) << Substitute(
+          "Error updating tablets from $0: $1. Tablet report was: $2",
+          ts_desc->permanent_uuid(), s.ToString(), SecureShortDebugString(full_report));
+      return s;
+    }
   }
 
   // Having successfully written the tablet mutations, this function cannot
@@ -4333,11 +4369,13 @@ Status CatalogManager::StoreLatestNotificationLogEventId(int64_t event_id) {
   DCHECK(hms_catalog_);
   DCHECK_GT(event_id, hms_notification_log_event_id_);
   leader_lock_.AssertAcquiredForReading();
-  SysCatalogTable::Actions actions;
-  actions.hms_notification_log_event_id = event_id;
-  RETURN_NOT_OK_PREPEND(
-      sys_catalog()->Write(actions),
-      "Failed to update processed Hive Metastore notification log ID in the sys catalog table");
+  {
+    SysCatalogTable::Actions actions;
+    actions.hms_notification_log_event_id = event_id;
+    RETURN_NOT_OK_PREPEND(sys_catalog()->Write(std::move(actions)),
+                          "Failed to update processed Hive Metastore "
+                          "notification log ID in the sys catalog table");
+  }
   hms_notification_log_event_id_ = event_id;
   return Status::OK();
 }
@@ -4587,13 +4625,15 @@ void CatalogManager::HandleTabletSchemaVersionReport(
   l.mutable_data()->set_state(SysTablesEntryPB::RUNNING,
                               Substitute("Current schema version=$0", current_version));
 
-  SysCatalogTable::Actions actions;
-  actions.table_to_update = table;
-  Status s = sys_catalog_->Write(actions);
-  if (!s.ok()) {
-    LOG_WITH_PREFIX(WARNING)
-        << "An error occurred while updating sys-tables: " << s.ToString();
-    return;
+  {
+    SysCatalogTable::Actions actions;
+    actions.table_to_update = table;
+    Status s = sys_catalog_->Write(std::move(actions));
+    if (PREDICT_FALSE(!s.ok())) {
+      LOG_WITH_PREFIX(WARNING)
+          << "An error occurred while updating sys-tables: " << s.ToString();
+      return;
+    }
   }
 
   l.Commit();
@@ -4671,11 +4711,13 @@ Status CatalogManager::ProcessPendingAssignments(
   }
 
   // Update the sys catalog with the new set of tablets/metadata.
-  SysCatalogTable::Actions actions;
-  actions.tablets_to_add = deferred.tablets_to_add;
-  actions.tablets_to_update = deferred.tablets_to_update;
-  RETURN_NOT_OK_PREPEND(sys_catalog_->Write(actions),
-                        "error persisting updated tablet metadata");
+  {
+    SysCatalogTable::Actions actions;
+    actions.tablets_to_add = deferred.tablets_to_add;
+    actions.tablets_to_update = deferred.tablets_to_update;
+    RETURN_NOT_OK_PREPEND(sys_catalog_->Write(std::move(actions)),
+                          "error persisting updated tablet metadata");
+  }
 
   // Expose tablet metadata changes before the new tablets themselves.
   lock_out.Commit();
@@ -4953,15 +4995,17 @@ Status CatalogManager::ReplaceTablet(const string& tablet_id, ReplaceTabletRespo
                                                              replace_msg);
 
   // Persist the changes to the syscatalog table.
-  SysCatalogTable::Actions actions;
-  actions.tablets_to_add.push_back(new_tablet);
-  actions.tablets_to_update.push_back(old_tablet);
-  Status s = sys_catalog_->Write(actions);
-  if (!s.ok()) {
-    s = s.CloneAndPrepend("an error occurred while writing to the sys-catalog");
-    LOG(WARNING) << s.ToString();
-    CheckIfNoLongerLeaderAndSetupError(s, resp);
-    return s;
+  {
+    SysCatalogTable::Actions actions;
+    actions.tablets_to_add.push_back(new_tablet);
+    actions.tablets_to_update.push_back(old_tablet);
+    Status s = sys_catalog_->Write(std::move(actions));
+    if (PREDICT_FALSE(!s.ok())) {
+      s = s.CloneAndPrepend("an error occurred while writing to the sys-catalog");
+      LOG(WARNING) << s.ToString();
+      CheckIfNoLongerLeaderAndSetupError(s, resp);
+      return s;
+    }
   }
 
   // Now commit the in-memory state and modify the global tablet map.
diff --git a/src/kudu/master/sys_catalog-test.cc b/src/kudu/master/sys_catalog-test.cc
index c67740c..6fbb942 100644
--- a/src/kudu/master/sys_catalog-test.cc
+++ b/src/kudu/master/sys_catalog-test.cc
@@ -131,7 +131,9 @@ static bool MetadatasEqual(const scoped_refptr<C>& ti_a,
 // visit)
 TEST_F(SysCatalogTest, TestSysCatalogTablesOperations) {
   TestTableLoader loader;
-  ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTables(&loader));
+  auto* sys_catalog = master_->catalog_manager()->sys_catalog();
+
+  ASSERT_OK(sys_catalog->VisitTables(&loader));
   ASSERT_EQ(0, loader.tables.size());
 
   // Create new table.
@@ -144,15 +146,17 @@ TEST_F(SysCatalogTest, TestSysCatalogTablesOperations) {
     l.mutable_data()->pb.set_state(SysTablesEntryPB::PREPARING);
     ASSERT_OK(SchemaToPB(Schema(), l.mutable_data()->pb.mutable_schema()));
     // Add the table
-    SysCatalogTable::Actions actions;
-    actions.table_to_add = table.get();
-    ASSERT_OK(master_->catalog_manager()->sys_catalog()->Write(actions));
+    {
+      SysCatalogTable::Actions actions;
+      actions.table_to_add = table.get();
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
     l.Commit();
   }
 
   // Verify it showed up.
   loader.Reset();
-  ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTables(&loader));
+  ASSERT_OK(sys_catalog->VisitTables(&loader));
   ASSERT_EQ(1, loader.tables.size());
   ASSERT_TRUE(MetadatasEqual(table, loader.tables[0]));
 
@@ -161,23 +165,27 @@ TEST_F(SysCatalogTest, TestSysCatalogTablesOperations) {
     TableMetadataLock l(table.get(), LockMode::WRITE);
     l.mutable_data()->pb.set_version(1);
     l.mutable_data()->pb.set_state(SysTablesEntryPB::REMOVED);
-    SysCatalogTable::Actions actions;
-    actions.table_to_update = table.get();
-    ASSERT_OK(master_->catalog_manager()->sys_catalog()->Write(actions));
+    {
+      SysCatalogTable::Actions actions;
+      actions.table_to_update = table.get();
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
     l.Commit();
   }
 
   loader.Reset();
-  ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTables(&loader));
+  ASSERT_OK(sys_catalog->VisitTables(&loader));
   ASSERT_EQ(1, loader.tables.size());
   ASSERT_TRUE(MetadatasEqual(table, loader.tables[0]));
 
   // Delete the table
   loader.Reset();
-  SysCatalogTable::Actions actions;
-  actions.table_to_delete = table.get();
-  ASSERT_OK(master_->catalog_manager()->sys_catalog()->Write(actions));
-  ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTables(&loader));
+  {
+    SysCatalogTable::Actions actions;
+    actions.table_to_delete = table.get();
+    ASSERT_OK(sys_catalog->Write(std::move(actions)));
+  }
+  ASSERT_OK(sys_catalog->VisitTables(&loader));
   ASSERT_EQ(0, loader.tables.size());
 }
 
@@ -275,9 +283,11 @@ TEST_F(SysCatalogTest, TestSysCatalogTabletsOperations) {
     loader.Reset();
     TabletMetadataLock l1(tablet1.get(), LockMode::WRITE);
     TabletMetadataLock l2(tablet2.get(), LockMode::WRITE);
-    SysCatalogTable::Actions actions;
-    actions.tablets_to_add = { tablet1, tablet2 };
-    ASSERT_OK(sys_catalog->Write(actions));
+    {
+      SysCatalogTable::Actions actions;
+      actions.tablets_to_add = { tablet1, tablet2 };
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
     l1.Commit();
     l2.Commit();
 
@@ -291,9 +301,11 @@ TEST_F(SysCatalogTest, TestSysCatalogTabletsOperations) {
   {
     TabletMetadataLock l1(tablet1.get(), LockMode::WRITE);
     l1.mutable_data()->pb.set_state(SysTabletsEntryPB::RUNNING);
-    SysCatalogTable::Actions actions;
-    actions.tablets_to_update = { tablet1 };
-    ASSERT_OK(sys_catalog->Write(actions));
+    {
+      SysCatalogTable::Actions actions;
+      actions.tablets_to_update = { tablet1 };
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
     l1.Commit();
 
     loader.Reset();
@@ -312,10 +324,12 @@ TEST_F(SysCatalogTest, TestSysCatalogTabletsOperations) {
     l2.mutable_data()->pb.set_state(SysTabletsEntryPB::RUNNING);
 
     loader.Reset();
-    SysCatalogTable::Actions actions;
-    actions.tablets_to_add = { tablet3 };
-    actions.tablets_to_update = { tablet1, tablet2 };
-    ASSERT_OK(sys_catalog->Write(actions));
+    {
+      SysCatalogTable::Actions actions;
+      actions.tablets_to_add = { tablet3 };
+      actions.tablets_to_update = { tablet1, tablet2 };
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
 
     l1.Commit();
     l2.Commit();
@@ -331,10 +345,12 @@ TEST_F(SysCatalogTest, TestSysCatalogTabletsOperations) {
   // Delete tablet1 and tablet3 tablets
   {
     loader.Reset();
-    SysCatalogTable::Actions actions;
-    actions.tablets_to_delete = { tablet1, tablet3 };
-    ASSERT_OK(master_->catalog_manager()->sys_catalog()->Write(actions));
-    ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTablets(&loader));
+    {
+      SysCatalogTable::Actions actions;
+      actions.tablets_to_delete = { tablet1, tablet3 };
+      ASSERT_OK(sys_catalog->Write(std::move(actions)));
+    }
+    ASSERT_OK(sys_catalog->VisitTablets(&loader));
     ASSERT_EQ(1, loader.tablets.size());
     ASSERT_TRUE(MetadatasEqual(tablet2, loader.tablets[0]));
   }
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index e82f20f..e716c23 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -525,7 +525,7 @@ Schema SysCatalogTable::BuildTableSchema() {
   return builder.Build();
 }
 
-Status SysCatalogTable::Write(const Actions& actions) {
+Status SysCatalogTable::Write(Actions actions, WriteMode mode) {
   TRACE_EVENT0("master", "SysCatalogTable::Write");
 
   WriteRequestPB req;
@@ -542,9 +542,23 @@ Status SysCatalogTable::Write(const Actions& actions) {
     ReqDeleteTable(&req, actions.table_to_delete);
   }
 
-  ReqAddTablets(&req, actions.tablets_to_add);
-  ReqUpdateTablets(&req, actions.tablets_to_update);
-  ReqDeleteTablets(&req, actions.tablets_to_delete);
+  // There might be many updates on tablet metadata in a cluster, especially
+  // in a big one. When persisting these, the write operations are broken into
+  // chunks if requested.
+  const size_t max_batch_size =
+      (mode == WriteMode::ATOMIC) ? 0 : GetMaxWriteRequestSize();
+  RETURN_NOT_OK(ChunkedWrite(&SysCatalogTable::ReqAddTablets,
+                             max_batch_size,
+                             std::move(actions.tablets_to_add),
+                             &req));
+  RETURN_NOT_OK(ChunkedWrite(&SysCatalogTable::ReqUpdateTablets,
+                             max_batch_size,
+                             std::move(actions.tablets_to_update),
+                             &req));
+  RETURN_NOT_OK(ChunkedWrite(&SysCatalogTable::ReqDeleteTablets,
+                             max_batch_size,
+                             std::move(actions.tablets_to_delete),
+                             &req));
 
   if (actions.hms_notification_log_event_id) {
     ReqSetNotificationLogEventId(&req, *actions.hms_notification_log_event_id);
@@ -558,9 +572,9 @@ Status SysCatalogTable::Write(const Actions& actions) {
   return SyncWrite(req);
 }
 
-// ==================================================================
+// ------------------------------------------------------------------
 // Table related methods
-// ==================================================================
+// ------------------------------------------------------------------
 
 void SysCatalogTable::ReqAddTable(WriteRequestPB* req,
                                   const scoped_refptr<TableInfo>& table) {
@@ -834,32 +848,49 @@ Status SysCatalogTable::RemoveTServerState(const string& tserver_id) {
   return SyncWrite(req);
 }
 
-// ==================================================================
+// ------------------------------------------------------------------
 // Tablet related methods
-// ==================================================================
-
-void SysCatalogTable::ReqAddTablets(WriteRequestPB* req,
-                                    const vector<scoped_refptr<TabletInfo>>& tablets) {
+// ------------------------------------------------------------------
+
+void SysCatalogTable::ReqAddTablets(
+    size_t max_size,
+    vector<scoped_refptr<TabletInfo>> tablets,
+    vector<scoped_refptr<TabletInfo>>* excess_tablets,
+    WriteRequestPB* req) {
+  DCHECK(excess_tablets);
   faststring metadata_buf;
   KuduPartialRow row(&schema_);
   RowOperationsPBEncoder enc(req->mutable_row_operations());
-  for (const auto& tablet : tablets) {
-    VLOG(2) << "Adding tablet " << tablet->id() << " in catalog: "
-            << SecureShortDebugString(tablet->metadata().dirty().pb);
+  size_t req_size = req->ByteSizeLong();
+  for (auto it = tablets.begin(); it != tablets.end(); ++it) {
+    const auto& tablet = *it;
     pb_util::SerializeToString(tablet->metadata().dirty().pb, &metadata_buf);
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
     CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->id()));
     CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
-    enc.Add(RowOperationsPB::INSERT, row);
+    req_size += enc.Add(RowOperationsPB::INSERT, row);
+    if (max_size > 0 && req_size > max_size && !enc.empty()) {
+      enc.RemoveLast();
+      std::move(it, tablets.end(), back_inserter(*excess_tablets));
+      break;
+    }
+    VLOG(2) << "Adding tablet " << tablet->id() << " in catalog: "
+            << SecureShortDebugString(tablet->metadata().dirty().pb);
   }
 }
 
-void SysCatalogTable::ReqUpdateTablets(WriteRequestPB* req,
-                                       const vector<scoped_refptr<TabletInfo>>& tablets) {
+void SysCatalogTable::ReqUpdateTablets(
+    size_t max_size,
+    vector<scoped_refptr<TabletInfo>> tablets,
+    vector<scoped_refptr<TabletInfo>>* excess_tablets,
+    WriteRequestPB* req) {
+  DCHECK(excess_tablets);
   faststring metadata_buf;
   KuduPartialRow row(&schema_);
   RowOperationsPBEncoder enc(req->mutable_row_operations());
-  for (const auto& tablet : tablets) {
+  size_t req_size = req->ByteSizeLong();
+  for (auto it = tablets.begin(); it != tablets.end(); ++it) {
+    const auto& tablet = *it;
     string diff;
     if (ArePBsEqual(tablet->metadata().state().pb,
                     tablet->metadata().dirty().pb,
@@ -867,27 +898,68 @@ void SysCatalogTable::ReqUpdateTablets(WriteRequestPB* req,
       // Short-circuit empty updates.
       continue;
     }
-    VLOG(2) << "Updating tablet " << tablet->id() << " in catalog: "
-            << diff;
+
     pb_util::SerializeToString(tablet->metadata().dirty().pb, &metadata_buf);
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
     CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->id()));
     CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
-    enc.Add(RowOperationsPB::UPDATE, row);
+    req_size += enc.Add(RowOperationsPB::UPDATE, row);
+    if (max_size > 0 && req_size > max_size && !enc.empty()) {
+      enc.RemoveLast();
+      std::move(it, tablets.end(), back_inserter(*excess_tablets));
+      break;
+    }
+    VLOG(2) << "Updating tablet " << tablet->id() << " in catalog: " << diff;
   }
 }
 
-void SysCatalogTable::ReqDeleteTablets(WriteRequestPB* req,
-                                       const vector<scoped_refptr<TabletInfo>>& tablets) {
+void SysCatalogTable::ReqDeleteTablets(
+    size_t max_size,
+    vector<scoped_refptr<TabletInfo>> tablets,
+    vector<scoped_refptr<TabletInfo>>* excess_tablets,
+    WriteRequestPB* req) {
+  DCHECK(excess_tablets);
   KuduPartialRow row(&schema_);
   RowOperationsPBEncoder enc(req->mutable_row_operations());
-  for (const auto& tablet : tablets) {
+  size_t req_size = req->ByteSizeLong();
+  for (auto it = tablets.begin(); it != tablets.end(); ++it) {
+    const auto& tablet = *it;
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
     CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->id()));
-    enc.Add(RowOperationsPB::DELETE, row);
+    req_size += enc.Add(RowOperationsPB::DELETE, row);
+    if (max_size > 0 && req_size > max_size && !enc.empty()) {
+      enc.RemoveLast();
+      std::move(it, tablets.end(), back_inserter(*excess_tablets));
+      break;
+    }
+    VLOG(2) << "Deleting tablet " << tablet->id() << " from catalog";
   }
 }
 
+Status SysCatalogTable::ChunkedWrite(
+    const Generator& generator,
+    size_t max_chunk_size,
+    vector<scoped_refptr<TabletInfo>> tablets_info,
+    WriteRequestPB* req) {
+  decltype(tablets_info) input(std::move(tablets_info));
+  do {
+    decltype(tablets_info) excess;
+    generator(*this, max_chunk_size, std::move(input), &excess, req);
+    if (!excess.empty()) {
+      // It's time to write a chunk of the generated data because
+      // the generator returned some of the input elements back. Those extra
+      // elements will go next batch if trying to stay under the specified
+      // maximum size for the result request.
+      RETURN_NOT_OK(SyncWrite(*req));
+      req->Clear();
+      req->set_tablet_id(kSysCatalogTabletId);
+      RETURN_NOT_OK(SchemaToPB(schema_, req->mutable_schema()));
+    }
+    input = std::move(excess);
+  } while (!input.empty());
+  return Status::OK();
+}
+
 void SysCatalogTable::ReqSetNotificationLogEventId(WriteRequestPB* req, int64_t event_id) {
   SysNotificationLogEventIdPB pb;
   pb.set_latest_notification_log_event_id(event_id);
diff --git a/src/kudu/master/sys_catalog.h b/src/kudu/master/sys_catalog.h
index 54b5fd6..0b31d1f 100644
--- a/src/kudu/master/sys_catalog.h
+++ b/src/kudu/master/sys_catalog.h
@@ -17,6 +17,7 @@
 #ifndef KUDU_MASTER_SYS_CATALOG_H_
 #define KUDU_MASTER_SYS_CATALOG_H_
 
+#include <cstddef>
 #include <cstdint>
 #include <functional>
 #include <set>
@@ -185,7 +186,25 @@ class SysCatalogTable {
     std::vector<scoped_refptr<TabletInfo>> tablets_to_delete;
     boost::optional<int64_t> hms_notification_log_event_id;
   };
-  Status Write(const Actions& actions);
+
+  // The way how actions are persisted into the system catalog table when
+  // calling the Write() method below.
+  enum class WriteMode {
+    // Write all the actions in a single atomic update to the system
+    // catalog tablet.
+    ATOMIC,
+
+    // Chunk the actions into pieces of maximum size corresponding to the
+    // maximum RPC size limit. This is to allow the leader replica of the system
+    // tablet to propagate the update to followers via Raft UpdateConsensus RPC.
+    CHUNKED,
+  };
+
+  // Persist the specified actions into the system tablet. Set the 'mode' to
+  // WriteMode::CHUNKED to split requests larger than the maximum RPC size
+  // into chunks if non-atomic update is acceptable. In case of chunked mode,
+  // no atomicity is guaranteed while persisting the specified actions.
+  Status Write(Actions actions, WriteMode mode = WriteMode::ATOMIC);
 
   // Scan of the table-related entries.
   Status VisitTables(TableVisitor* visitor);
@@ -292,12 +311,45 @@ class SysCatalogTable {
                       const scoped_refptr<TableInfo>& table);
   void ReqDeleteTable(tserver::WriteRequestPB* req,
                       const scoped_refptr<TableInfo>& table);
-  void ReqAddTablets(tserver::WriteRequestPB* req,
-                     const std::vector<scoped_refptr<TabletInfo>>& tablets);
-  void ReqUpdateTablets(tserver::WriteRequestPB* req,
-                        const std::vector<scoped_refptr<TabletInfo>>& tablets);
-  void ReqDeleteTablets(tserver::WriteRequestPB* req,
-                        const std::vector<scoped_refptr<TabletInfo>>& tablets);
+
+  // These three methods below generate WriteRequestPB to persist the
+  // information on the tablet metadata updates in the system catalog. The size
+  // of the generated write request is limited by the 'max_size' parameter.
+  // The information on the tablets' metadata updates is provided with the
+  // 'tablets' parameter. The 'excess_tablets' output parameter is populated
+  // with elements which didn't fit into the result request due to the sizing
+  // limitations. The result request is returned via the 'req' output parameter.
+  // Note the best effort behavior of these methods: the resulting request
+  // cannot be empty, and the very first row might be over the limit already.
+  // However, that will be caught later by SyncWrite() before actually writing
+  // the generated data into the system tablet.
+  void ReqAddTablets(size_t max_size,
+                     std::vector<scoped_refptr<TabletInfo>> tablets,
+                     std::vector<scoped_refptr<TabletInfo>>* excess_tablets,
+                     tserver::WriteRequestPB* req);
+  void ReqUpdateTablets(size_t max_size,
+                        std::vector<scoped_refptr<TabletInfo>> tablets,
+                        std::vector<scoped_refptr<TabletInfo>>* excess_tablets,
+                        tserver::WriteRequestPB* req);
+  void ReqDeleteTablets(size_t max_size,
+                        std::vector<scoped_refptr<TabletInfo>> tablets,
+                        std::vector<scoped_refptr<TabletInfo>>* excess_tablets,
+                        tserver::WriteRequestPB* req);
+
+  // Generator function used by the ChunkedWrite() method below. The three
+  // methods above (ReqAddTablets, ReqUpdateTablets, and ReqDeleteTablets)
+  // are the generators with corresponding signature.
+  typedef std::function<void(SysCatalogTable&,
+                             size_t,
+                             std::vector<scoped_refptr<TabletInfo>>,
+                             std::vector<scoped_refptr<TabletInfo>>*,
+                             tserver::WriteRequestPB*)> Generator;
+
+  // A method for chunked write into the system tablet.
+  Status ChunkedWrite(const Generator& generator,
+                      size_t max_chunk_size,
+                      std::vector<scoped_refptr<TabletInfo>> tablets_info,
+                      tserver::WriteRequestPB* req);
 
   // Overwrite (upsert) the latest event ID in the table with the provided ID.
   void ReqSetNotificationLogEventId(tserver::WriteRequestPB* req, int64_t event_id);