You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2016/12/07 19:49:34 UTC

[1/2] kudu git commit: KUDU-1792 It is possible to overwrite range partitions

Repository: kudu
Updated Branches:
  refs/heads/master 93649c469 -> a70c90500


KUDU-1792 It is possible to overwrite range partitions

This fixes another bug (see KUDU-1750) in the master code which detects
overlaps in range partitions during alter table operations. A new test
is included which attempts to exhaustively cover all cases of conflict
between pairs of ranges being added. It also adds a lot of coverage for
drop conflicts between pairs of ranges, although the coverage is not
intended to be exhaustive.

The next step if we want to test this space even more would be to
increase the number of ranges involved in conflicts to three or more,
but since this amplifies the search space so much, it is probably best
done as part of a randomized fuzz test.

Change-Id: I89ae162f33de20169848091e518315853be30460
Reviewed-on: http://gerrit.cloudera.org:8080/5395
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/bd72a38c
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/bd72a38c
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/bd72a38c

Branch: refs/heads/master
Commit: bd72a38cd144a8beb1b1a320010c19adb9ca1eee
Parents: 93649c4
Author: Dan Burkert <da...@apache.org>
Authored: Tue Dec 6 17:08:30 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Wed Dec 7 19:48:48 2016 +0000

----------------------------------------------------------------------
 python/kudu/tests/test_client.py               |   6 +
 src/kudu/integration-tests/alter_table-test.cc | 315 ++++++++++++++++++++
 src/kudu/master/catalog_manager.cc             |  12 +-
 3 files changed, 329 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/bd72a38c/python/kudu/tests/test_client.py
----------------------------------------------------------------------
diff --git a/python/kudu/tests/test_client.py b/python/kudu/tests/test_client.py
index 89d5b62..3183051 100644
--- a/python/kudu/tests/test_client.py
+++ b/python/kudu/tests/test_client.py
@@ -349,6 +349,9 @@ class TestClient(KuduTestBase, unittest.TestCase):
         # Add Range Partition
         table = self.client.table(self.ex_table)
         alterer = self.client.new_table_alterer(table)
+        # Drop the unbounded range partition.
+        alterer.drop_range_partition()
+        # Add a partition from 0 to 100
         alterer.add_range_partition(
             lower_bound={'key': 0},
             upper_bound={'key': 100}
@@ -357,10 +360,13 @@ class TestClient(KuduTestBase, unittest.TestCase):
         # TODO(jtbirdsell): Once C++ client can list partition schema
         # then this test should confirm that the partition was added.
         alterer = self.client.new_table_alterer(table)
+        # Drop the partition from 0 to 100
         alterer.drop_range_partition(
             lower_bound={'key': 0},
             upper_bound={'key': 100}
         )
+        # Add back the unbounded range partition
+        alterer.add_range_partition()
         table = alterer.alter()
 
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd72a38c/src/kudu/integration-tests/alter_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index e28f8bf..6ae9be2 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -22,6 +22,7 @@
 #include <utility>
 
 #include <boost/bind.hpp>
+#include <boost/optional.hpp>
 #include <gflags/gflags.h>
 #include <gtest/gtest.h>
 
@@ -1503,6 +1504,320 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   ASSERT_EQ(100, CountTableRows(table.get()));
 }
 
+// Attempts to exhaustively check all cases of single-column range partition
+// conflicts for ALTER TABLE ADD RANGE PARTITION ops involving two ranges.
+//
+// Also tests some cases of DROP RANGE PARTITION where possible, but the
+// coverage is not exhaustive (the state space for invalid add/drop combinations
+// is much bigger than for add/add combinations).
+//
+// Regression test for KUDU-1792
+TEST_F(AlterTableTest, TestAddRangePartitionConflictExhaustive) {
+  unique_ptr<KuduTableAlterer> table_alterer;
+
+  // CREATE TABLE t (c0 INT PRIMARY KEY)
+  // PARTITION BY
+  //    RANGE (c0) ();
+  string table_name = "test-alter-range-partitioning-invalid-unbounded";
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  ASSERT_OK(table_creator->table_name(table_name)
+                          .schema(&schema_)
+                          .set_range_partition_columns({ "c0" })
+                          .num_replicas(1)
+                          .Create());
+  shared_ptr<KuduTable> table;
+  ASSERT_OK(client_->OpenTable(table_name, &table));
+
+  // Drop the default UNBOUNDED tablet in order to start with a table with no ranges.
+  table_alterer.reset(client_->NewTableAlterer(table_name));
+  ASSERT_OK(table_alterer->DropRangePartition(schema_.NewRow(), schema_.NewRow())
+                         ->wait(true)->Alter());
+
+  // Turns an optional value into a row for the table.
+  auto fill_row = [&] (boost::optional<int32_t> value) -> unique_ptr<KuduPartialRow> {
+    unique_ptr<KuduPartialRow> row(schema_.NewRow());
+    if (value) {
+      CHECK_OK(row->SetInt32("c0", *value));
+    }
+    return row;
+  };
+
+  // Attempts to add a range partition to the table with the specified bounds.
+  auto add_range_partition = [&] (boost::optional<int32_t> lower_bound,
+                                  boost::optional<int32_t> upper_bound) -> Status {
+    table_alterer.reset(client_->NewTableAlterer(table_name));
+    return table_alterer->AddRangePartition(fill_row(lower_bound).release(),
+                                            fill_row(upper_bound).release())
+                        ->wait(false)
+                        ->Alter();
+  };
+
+  // Attempts to drop a range partition to the table with the specified bounds.
+  auto drop_range_partition = [&] (boost::optional<int32_t> lower_bound,
+                                   boost::optional<int32_t> upper_bound) -> Status {
+    table_alterer.reset(client_->NewTableAlterer(table_name));
+    return table_alterer->DropRangePartition(fill_row(lower_bound).release(),
+                                             fill_row(upper_bound).release())
+                        ->wait(false)
+                        ->Alter();
+  };
+
+  // Attempts to add two range partitions to the table in a single transaction.
+  auto add_range_partitions = [&] (boost::optional<int32_t> a_lower_bound,
+                                   boost::optional<int32_t> a_upper_bound,
+                                   boost::optional<int32_t> b_lower_bound,
+                                   boost::optional<int32_t> b_upper_bound) -> Status {
+    table_alterer.reset(client_->NewTableAlterer(table_name));
+    return table_alterer->AddRangePartition(fill_row(a_lower_bound).release(),
+                                            fill_row(a_upper_bound).release())
+                        ->AddRangePartition(fill_row(b_lower_bound).release(),
+                                            fill_row(b_upper_bound).release())
+                        ->wait(false)
+                        ->Alter();
+  };
+
+  // Attempts to add and drop two range partitions in a single transaction.
+  auto add_drop_range_partitions = [&] (boost::optional<int32_t> a_lower_bound,
+                                        boost::optional<int32_t> a_upper_bound,
+                                        boost::optional<int32_t> b_lower_bound,
+                                        boost::optional<int32_t> b_upper_bound) -> Status {
+    table_alterer.reset(client_->NewTableAlterer(table_name));
+    return table_alterer->AddRangePartition(fill_row(a_lower_bound).release(),
+                                            fill_row(a_upper_bound).release())
+                        ->DropRangePartition(fill_row(b_lower_bound).release(),
+                                             fill_row(b_upper_bound).release())
+                        ->wait(false)
+                        ->Alter();
+  };
+
+  auto bounds_to_string = [] (boost::optional<int32_t> lower_bound,
+                              boost::optional<int32_t> upper_bound) -> string {
+    if (!lower_bound && !upper_bound) {
+      return "UNBOUNDED";
+    }
+    if (!lower_bound) {
+      return strings::Substitute("VALUES < $0", *upper_bound);
+    }
+    if (!upper_bound) {
+      return strings::Substitute("VALUES >= $0", *lower_bound);
+    }
+    return strings::Substitute("$0 <= VALUES < $1", *lower_bound, *upper_bound);
+  };
+
+  // Checks that b conflicts with a, when added in that order.
+  auto do_expect_range_partitions_conflict = [&] (boost::optional<int32_t> a_lower_bound,
+                                                  boost::optional<int32_t> a_upper_bound,
+                                                  boost::optional<int32_t> b_lower_bound,
+                                                  boost::optional<int32_t> b_upper_bound) {
+    SCOPED_TRACE(strings::Substitute("b: $0", bounds_to_string(b_lower_bound, b_upper_bound)));
+    SCOPED_TRACE(strings::Substitute("a: $0", bounds_to_string(a_lower_bound, a_upper_bound)));
+
+    // Add a then add b.
+    ASSERT_OK(add_range_partition(a_lower_bound, a_upper_bound));
+    Status s = add_range_partition(b_lower_bound, b_upper_bound);
+    ASSERT_FALSE(s.ok());
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "New range partition conflicts with existing range partition");
+    // Clean up by removing a.
+    ASSERT_OK(drop_range_partition(a_lower_bound, a_upper_bound));
+
+    // Add a and b in the same transaction.
+    s = add_range_partitions(a_lower_bound, a_upper_bound,
+                             b_lower_bound, b_upper_bound);
+    ASSERT_FALSE(s.ok());
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "New range partition conflicts with another new range partition");
+
+    // To get some extra coverage of DROP RANGE PARTITION, check if the two
+    // ranges are not equal, and if so, check that adding one and dropping the
+    // other fails.
+
+    if (a_lower_bound != b_lower_bound || a_upper_bound != b_upper_bound) {
+      // Add a then drop b.
+      ASSERT_OK(add_range_partition(a_lower_bound, a_upper_bound));
+      Status s = drop_range_partition(b_lower_bound, b_upper_bound);
+      ASSERT_FALSE(s.ok());
+      ASSERT_STR_CONTAINS(s.ToString(), "No range partition found for drop range partition step");
+      // Clean up by removing a.
+      ASSERT_OK(drop_range_partition(a_lower_bound, a_upper_bound));
+
+      // Add a and drop b in a single transaction.
+      s = add_drop_range_partitions(a_lower_bound, a_upper_bound,
+                                    b_lower_bound, b_upper_bound);
+      ASSERT_FALSE(s.ok());
+      ASSERT_STR_CONTAINS(s.ToString(),
+                          "No range partition found for drop range partition step");
+    }
+  };
+
+  // Checks that two range partitions conflict.
+  auto expect_range_partitions_conflict = [&] (boost::optional<int32_t> a_lower_bound,
+                                               boost::optional<int32_t> a_upper_bound,
+                                               boost::optional<int32_t> b_lower_bound,
+                                               boost::optional<int32_t> b_upper_bound) {
+    do_expect_range_partitions_conflict(a_lower_bound, a_upper_bound,
+                                        b_lower_bound, b_upper_bound);
+    do_expect_range_partitions_conflict(b_lower_bound, b_upper_bound,
+                                        a_lower_bound, a_upper_bound);
+  };
+
+  /// Bounded / Bounded
+
+  // [----------)
+  // [----------)
+  expect_range_partitions_conflict(0, 100, 0, 100);
+
+  // [----------)
+  //      [----------)
+  expect_range_partitions_conflict(0, 100, 50, 150);
+
+  // [----------)
+  // [------)
+  expect_range_partitions_conflict(0, 100, 0, 50);
+
+  // [----------)
+  //     [------)
+  expect_range_partitions_conflict(0, 100, 50, 100);
+
+  // [----------)
+  //   [------)
+  expect_range_partitions_conflict(0, 100, 25, 75);
+
+  /// Bounded / Unbounded Above
+
+  //  [----------)
+  // [-------------->
+  expect_range_partitions_conflict(0, 100, -1, boost::none);
+
+  // [----------)
+  // [-------------->
+  expect_range_partitions_conflict(0, 100, 0, boost::none);
+
+  // [----------)
+  //  [------------->
+  expect_range_partitions_conflict(0, 100, 1, boost::none);
+
+  // [----------)
+  //      [------------->
+  expect_range_partitions_conflict(0, 100, 50, boost::none);
+
+  // [----------)
+  //           [--------->
+  expect_range_partitions_conflict(0, 100, 99, boost::none);
+
+  /// Bounded / Unbounded Below
+
+  //        [----------)
+  // <-------)
+  expect_range_partitions_conflict(0, 100, boost::none, 1);
+
+  //        [----------)
+  // <------------)
+  expect_range_partitions_conflict(0, 100, boost::none, 50);
+
+  //        [----------)
+  // <-----------------)
+  expect_range_partitions_conflict(0, 100, boost::none, 100);
+
+  //        [----------)
+  // <-------------------)
+  expect_range_partitions_conflict(0, 100, boost::none, 125);
+
+  /// Bounded / Unbounded
+
+  //     [----------)
+  // <------------------->
+  expect_range_partitions_conflict(0, 100, boost::none, boost::none);
+
+  /// Bounded / Single Value
+
+  // [----------)
+  // |
+  expect_range_partitions_conflict(0, 100, 0, 1);
+
+  // [----------)
+  //      |
+  expect_range_partitions_conflict(0, 100, 25, 26);
+
+  // [----------)
+  //           |
+  expect_range_partitions_conflict(0, 100, 99, 100);
+
+  /// Unbounded Above / Unbounded Above
+
+  //    [---------->
+  // [---------->
+  expect_range_partitions_conflict(0, boost::none, -10, boost::none);
+
+  // [---------->
+  // [---------->
+  expect_range_partitions_conflict(0, boost::none, 0, boost::none);
+
+  /// Unbounded Above / Unbounded Below
+
+  // [---------->
+  // <----------)
+  expect_range_partitions_conflict(0, boost::none, boost::none, 100);
+
+  //        [---------->
+  // <-------)
+  expect_range_partitions_conflict(0, boost::none, boost::none, 1);
+
+  /// Unbounded Above / Unbounded
+
+  // [---------->
+  // <---------->
+  expect_range_partitions_conflict(0, boost::none, boost::none, boost::none);
+
+  /// Unbounded Above / Single Value
+
+  // [---------->
+  // |
+  expect_range_partitions_conflict(0, boost::none, 0, 1);
+
+  // [---------->
+  //   |
+  expect_range_partitions_conflict(0, boost::none, 100, 101);
+
+  /// Unbounded Below / Unbounded Below
+
+  // <----------)
+  // <----------)
+  expect_range_partitions_conflict(boost::none, 100, boost::none, 100);
+
+  // <----------)
+  // <-----)
+  expect_range_partitions_conflict(boost::none, 100, boost::none, 50);
+
+  /// Unbounded Below / Unbounded
+
+  // <----------)
+  // <---------->
+  expect_range_partitions_conflict(boost::none, 100, boost::none, boost::none);
+
+  /// Unbounded Below / Single Value
+
+  // <----------)
+  //       |
+  expect_range_partitions_conflict(boost::none, 100, 50, 51);
+
+  // <----------)
+  //           |
+  expect_range_partitions_conflict(boost::none, 100, 99, 100);
+
+  /// Unbounded / Unbounded
+
+  // <---------->
+  // <---------->
+  expect_range_partitions_conflict(boost::none, boost::none, boost::none, boost::none);
+
+  /// Single Value / Single Value
+
+  // |
+  // |
+  expect_range_partitions_conflict(0, 1, 0, 1);
+}
+
 TEST_F(ReplicatedAlterTableTest, TestReplicatedAlter) {
   const int kNumRows = 100;
   InsertRows(0, kNumRows);

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd72a38c/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 3df0d46..1e7fb68 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1451,7 +1451,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           auto existing_iter = existing_tablets.upper_bound(lower_bound);
           if (existing_iter != existing_tablets.end()) {
             TabletMetadataLock metadata(existing_iter->second, TabletMetadataLock::READ);
-            if (metadata.data().pb.partition().partition_key_start() < upper_bound) {
+            if (upper_bound.empty() ||
+                metadata.data().pb.partition().partition_key_start() < upper_bound) {
               return Status::InvalidArgument(
                   "New range partition conflicts with existing range partition",
                   partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
@@ -1459,7 +1460,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           }
           if (existing_iter != existing_tablets.begin()) {
             TabletMetadataLock metadata(std::prev(existing_iter)->second, TabletMetadataLock::READ);
-            if (metadata.data().pb.partition().partition_key_end() > lower_bound) {
+            if (metadata.data().pb.partition().partition_key_end().empty() ||
+                metadata.data().pb.partition().partition_key_end() > lower_bound) {
               return Status::InvalidArgument(
                   "New range partition conflicts with existing range partition",
                   partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
@@ -1470,7 +1472,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           auto new_iter = new_tablets.upper_bound(lower_bound);
           if (new_iter != new_tablets.end()) {
             const auto& metadata = new_iter->second->mutable_metadata()->dirty();
-            if (metadata.pb.partition().partition_key_start() < upper_bound) {
+            if (upper_bound.empty() ||
+                metadata.pb.partition().partition_key_start() < upper_bound) {
               return Status::InvalidArgument(
                   "New range partition conflicts with another new range partition",
                   partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
@@ -1478,7 +1481,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           }
           if (new_iter != new_tablets.begin()) {
             const auto& metadata = std::prev(new_iter)->second->mutable_metadata()->dirty();
-            if (metadata.pb.partition().partition_key_end() > lower_bound) {
+            if (metadata.pb.partition().partition_key_end().empty() ||
+                metadata.pb.partition().partition_key_end() > lower_bound) {
               return Status::InvalidArgument(
                   "New range partition conflicts with another new range partition",
                   partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));


[2/2] kudu git commit: Reduce default client tablet locations cache TTL

Posted by da...@apache.org.
Reduce default client tablet locations cache TTL

This reduces the default client tablet locations cache TTL from 60 to 5
minutes. This is expected to make default behavior much less surprising
for users who add and drop range partitions, with little impact on
master load or performance.

Change-Id: I3f93005f067ccd2d28b259e9483dd9384c546152
Reviewed-on: http://gerrit.cloudera.org:8080/5397
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/a70c9050
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/a70c9050
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/a70c9050

Branch: refs/heads/master
Commit: a70c9050069863cf64a1db9ab5ab6da3de4e0504
Parents: bd72a38
Author: Dan Burkert <da...@apache.org>
Authored: Tue Dec 6 21:19:36 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Wed Dec 7 19:49:08 2016 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/kudu/client/AlterTableOptions.java  | 4 ++--
 python/kudu/client.pyx                                           | 2 +-
 src/kudu/client/client.h                                         | 2 +-
 src/kudu/master/catalog_manager.cc                               | 2 +-
 4 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a70c9050/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
----------------------------------------------------------------------
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 72c70e7..31617aa 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
@@ -141,7 +141,7 @@ public class AlterTableOptions {
    * This client will immediately be able to write and scan the new tablets when the alter table
    * operation returns success, however other existing clients may have to wait for a timeout period
    * to elapse before the tablets become visible. This period is configured by the master's
-   * 'table_locations_ttl_ms' flag, and defaults to one hour.
+   * 'table_locations_ttl_ms' flag, and defaults to 5 minutes.
    *
    * @param lowerBound inclusive lower bound, may be empty but not null
    * @param upperBound exclusive upper bound, may be empty but not null
@@ -167,7 +167,7 @@ public class AlterTableOptions {
    * This client will immediately be able to write and scan the new tablets when the alter table
    * operation returns success, however other existing clients may have to wait for a timeout period
    * to elapse before the tablets become visible. This period is configured by the master's
-   * 'table_locations_ttl_ms' flag, and defaults to one hour.
+   * 'table_locations_ttl_ms' flag, and defaults to 5 minutes.
    *
    * @param lowerBound lower bound, may be empty but not null
    * @param upperBound upper bound, may be empty but not null

http://git-wip-us.apache.org/repos/asf/kudu/blob/a70c9050/python/kudu/client.pyx
----------------------------------------------------------------------
diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index 9492d2e..ab6a70a 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -2693,7 +2693,7 @@ cdef class TableAlterer:
         returns success, however other existing clients may have to wait for a
         timeout period to elapse before the tablets become visible. This period
         is configured by the master's 'table_locations_ttl_ms' flag, and
-        defaults to one hour.
+        defaults to 5 minutes.
 
         Parameters
         ----------

http://git-wip-us.apache.org/repos/asf/kudu/blob/a70c9050/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 2472254..4030638 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -984,7 +984,7 @@ class KUDU_EXPORT KuduTableAlterer {
   ///   Alter() returns success, however other existing clients may have to wait
   ///   for a timeout period to elapse before the tablets become visible. This
   ///   period is configured by the master's 'table_locations_ttl_ms' flag, and
-  ///   defaults to one hour.
+  ///   defaults to 5 minutes.
   ///
   /// @param [in] lower_bound
   ///   The lower bound of the range partition to add. If the row is empty, then

http://git-wip-us.apache.org/repos/asf/kudu/blob/a70c9050/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 1e7fb68..d42dc25 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -173,7 +173,7 @@ DEFINE_bool(catalog_manager_check_ts_count_for_create_table, true,
             "a table to be created.");
 TAG_FLAG(catalog_manager_check_ts_count_for_create_table, hidden);
 
-DEFINE_int32(table_locations_ttl_ms, 60 * 60 * 1000, // 1 hour
+DEFINE_int32(table_locations_ttl_ms, 5 * 60 * 1000, // 5 minutes
              "Maximum time in milliseconds which clients may cache table locations. "
              "New range partitions may not be visible to existing client instances "
              "until after waiting for the ttl period.");