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 2017/05/11 18:23:09 UTC

[1/4] kudu git commit: KUDU-1056 and KUDU-1020 Safe time for ksck checksum

Repository: kudu
Updated Branches:
  refs/heads/master ae7cfa04a -> de1daf282


KUDU-1056 and KUDU-1020 Safe time for ksck checksum

Now that safe time works properly, this patch enables
the snapshot checksum tests for ksck.

Change-Id: Ib45be20dcfa37fb85185302adf84d2c4a55f8c1e
Reviewed-on: http://gerrit.cloudera.org:8080/6843
Reviewed-by: David Ribeiro Alves <da...@gmail.com>
Tested-by: Will Berkeley <wd...@gmail.com>


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

Branch: refs/heads/master
Commit: e6141a0adf6e3c5a54be8cfdf5acd0f1ff65f714
Parents: ae7cfa0
Author: Will Berkeley <wd...@apache.org>
Authored: Tue May 9 14:28:06 2017 -0700
Committer: David Ribeiro Alves <da...@gmail.com>
Committed: Thu May 11 04:18:44 2017 +0000

----------------------------------------------------------------------
 src/kudu/tools/ksck.cc             |  6 +----
 src/kudu/tools/ksck_remote-test.cc | 42 +++++++++++++++------------------
 2 files changed, 20 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e6141a0a/src/kudu/tools/ksck.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/ksck.cc b/src/kudu/tools/ksck.cc
index 67beac1..cdec7a6 100644
--- a/src/kudu/tools/ksck.cc
+++ b/src/kudu/tools/ksck.cc
@@ -537,11 +537,7 @@ Status Ksck::ChecksumData(const ChecksumOptions& opts) {
                     num_results, num_tablet_replicas);
 
   if (num_mismatches != 0) {
-    // TODO(KUDU-1020): remove the below note once safe time advancement is fully implemented.
-    return Status::Corruption(Substitute(
-        "$0 checksum mismatches were detected. "
-        "NOTE: if the table is actively being written to, this may generate spurious "
-        "checksum mismatches.", num_mismatches));
+    return Status::Corruption(Substitute("$0 checksum mismatches were detected.", num_mismatches));
   }
   if (num_errors != 0) {
     return Status::Aborted(Substitute("$0 errors were detected", num_errors));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6141a0a/src/kudu/tools/ksck_remote-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/ksck_remote-test.cc b/src/kudu/tools/ksck_remote-test.cc
index 00591af..d57a401 100644
--- a/src/kudu/tools/ksck_remote-test.cc
+++ b/src/kudu/tools/ksck_remote-test.cc
@@ -151,7 +151,12 @@ class RemoteKsckTest : public KuduTest {
       if (!status.ok()) {
         promise->Set(status);
       }
-      started_writing->CountDown(1);
+      // Wait for the first 100 writes so that it's very likely all replicas have committed a
+      // message in each tablet; otherwise, safe time might not have been updated on all replicas
+      // and some might refuse snapshot scans because of lag.
+      if (i > 100) {
+        started_writing->CountDown(1);
+      }
     }
     promise->Set(Status::OK());
   }
@@ -231,7 +236,9 @@ TEST_F(RemoteKsckTest, TestChecksum) {
   MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(30);
   Status s;
   while (MonoTime::Now() < deadline) {
+    ASSERT_OK(ksck_->CheckMasterRunning());
     ASSERT_OK(ksck_->FetchTableAndTabletInfo());
+    ASSERT_OK(ksck_->FetchInfoFromTabletServers());
 
     err_stream_.str("");
     s = ksck_->ChecksumData(ChecksumOptions(MonoDelta::FromSeconds(1), 16, false, 0));
@@ -254,7 +261,9 @@ TEST_F(RemoteKsckTest, TestChecksumTimeout) {
   uint64_t num_writes = 10000;
   LOG(INFO) << "Generating row writes...";
   ASSERT_OK(GenerateRowWrites(num_writes));
+  ASSERT_OK(ksck_->CheckMasterRunning());
   ASSERT_OK(ksck_->FetchTableAndTabletInfo());
+  ASSERT_OK(ksck_->FetchInfoFromTabletServers());
   // Use an impossibly low timeout value of zero!
   Status s = ksck_->ChecksumData(ChecksumOptions(MonoDelta::FromNanoseconds(0), 16, false, 0));
   ASSERT_TRUE(s.IsTimedOut()) << "Expected TimedOut Status, got: " << s.ToString();
@@ -273,45 +282,32 @@ TEST_F(RemoteKsckTest, TestChecksumSnapshot) {
   CHECK(started_writing.WaitFor(MonoDelta::FromSeconds(30)));
 
   uint64_t ts = client_->GetLatestObservedTimestamp();
-  MonoTime start(MonoTime::Now());
-  MonoTime deadline = start + MonoDelta::FromSeconds(30);
-  Status s;
-  // TODO: We need to loop here because safe time is not yet implemented.
-  // Remove this loop when that is done. See KUDU-1056.
-  while (true) {
-    ASSERT_OK(ksck_->FetchTableAndTabletInfo());
-    Status s = ksck_->ChecksumData(ChecksumOptions(MonoDelta::FromSeconds(10), 16, true, ts));
-    if (s.ok()) break;
-    if (MonoTime::Now() > deadline) break;
-    SleepFor(MonoDelta::FromMilliseconds(10));
-  }
-  if (!s.ok()) {
-    LOG(WARNING) << Substitute("Timed out after $0 waiting for ksck to become consistent on TS $1. "
-                               "Status: $2",
-                               (MonoTime::Now() - start).ToString(),
-                               ts, s.ToString());
-    EXPECT_OK(s); // To avoid ASAN complaints due to thread reading the CountDownLatch.
-  }
+  ASSERT_OK(ksck_->CheckMasterRunning());
+  ASSERT_OK(ksck_->FetchTableAndTabletInfo());
+  ASSERT_OK(ksck_->FetchInfoFromTabletServers());
+  ASSERT_OK(ksck_->ChecksumData(ChecksumOptions(MonoDelta::FromSeconds(10), 16, true, ts)));
   continue_writing.Store(false);
   ASSERT_OK(promise.Get());
   writer_thread->Join();
 }
 
 // Test that followers & leader wait until safe time to respond to a snapshot
-// scan at current timestamp. TODO: Safe time not yet implemented. See KUDU-1056.
-TEST_F(RemoteKsckTest, DISABLED_TestChecksumSnapshotCurrentTimestamp) {
+// scan at current timestamp.
+TEST_F(RemoteKsckTest, TestChecksumSnapshotCurrentTimestamp) {
   CountDownLatch started_writing(1);
   AtomicBool continue_writing(true);
   Promise<Status> promise;
   scoped_refptr<Thread> writer_thread;
 
-  Thread::Create("RemoteKsckTest", "TestChecksumSnapshot",
+  Thread::Create("RemoteKsckTest", "TestChecksumSnapshotCurrentTimestamp",
                  &RemoteKsckTest::GenerateRowWritesLoop, this,
                  &started_writing, boost::cref(continue_writing), &promise,
                  &writer_thread);
   CHECK(started_writing.WaitFor(MonoDelta::FromSeconds(30)));
 
+  ASSERT_OK(ksck_->CheckMasterRunning());
   ASSERT_OK(ksck_->FetchTableAndTabletInfo());
+  ASSERT_OK(ksck_->FetchInfoFromTabletServers());
   ASSERT_OK(ksck_->ChecksumData(ChecksumOptions(MonoDelta::FromSeconds(10), 16, true,
                                                 ChecksumOptions::kCurrentTimestamp)));
   continue_writing.Store(false);


[3/4] kudu git commit: faststring: add shrink_to_fit()

Posted by al...@apache.org.
faststring: add shrink_to_fit()

This adds a shrink_to_fit() method which reallocates the underlying buffer of a
faststring to match its current length. This is useful in the case where a
faststring acts as a long-lived buffer which occasionally gets large values,
but often contains small ones.

Change-Id: I0e437ff180fccd1957d252fb9a3551bb91ba7917
Reviewed-on: http://gerrit.cloudera.org:8080/6835
Reviewed-by: David Ribeiro Alves <da...@gmail.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/dce25916
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/dce25916
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/dce25916

Branch: refs/heads/master
Commit: dce25916f428659072fd1e140a9081829956468e
Parents: 43fbfdf
Author: Todd Lipcon <to...@cloudera.com>
Authored: Tue May 9 14:30:41 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu May 11 16:25:10 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/CMakeLists.txt     |  1 +
 src/kudu/util/faststring-test.cc | 60 +++++++++++++++++++++++++++++++++++
 src/kudu/util/faststring.cc      | 21 ++++++++++--
 src/kudu/util/faststring.h       | 20 ++++++++++--
 4 files changed, 96 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/dce25916/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index d9a37cd..e4fff0c 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -339,6 +339,7 @@ ADD_KUDU_TEST(env-test LABELS no_tsan)
 ADD_KUDU_TEST(env_util-test)
 ADD_KUDU_TEST(errno-test)
 ADD_KUDU_TEST(failure_detector-test)
+ADD_KUDU_TEST(faststring-test)
 ADD_KUDU_TEST(file_cache-test)
 ADD_KUDU_TEST(file_cache-stress-test RUN_SERIAL true)
 ADD_KUDU_TEST(flag_tags-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/dce25916/src/kudu/util/faststring-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/faststring-test.cc b/src/kudu/util/faststring-test.cc
new file mode 100644
index 0000000..c57cb41
--- /dev/null
+++ b/src/kudu/util/faststring-test.cc
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <algorithm>
+#include "kudu/util/faststring.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+class FaststringTest : public KuduTest {};
+
+TEST_F(FaststringTest, TestShrinkToFit_Empty) {
+  faststring s;
+  s.shrink_to_fit();
+  ASSERT_EQ(faststring::kInitialCapacity, s.capacity());
+}
+
+// Test that, if the string contents is shorter than the initial capacity
+// of the faststring, shrink_to_fit() leaves the string in the built-in
+// array.
+TEST_F(FaststringTest, TestShrinkToFit_SmallerThanInitialCapacity) {
+  faststring s;
+  s.append("hello");
+  s.shrink_to_fit();
+  ASSERT_EQ(faststring::kInitialCapacity, s.capacity());
+}
+
+TEST_F(FaststringTest, TestShrinkToFit_Random) {
+  Random r(GetRandomSeed32());
+  int kMaxSize = faststring::kInitialCapacity * 2;
+  std::unique_ptr<char[]> random_bytes(new char[kMaxSize]);
+  RandomString(random_bytes.get(), kMaxSize, &r);
+
+  faststring s;
+  for (int i = 0; i < 100; i++) {
+    int new_size = r.Uniform(kMaxSize);
+    s.resize(new_size);
+    memcpy(s.data(), random_bytes.get(), new_size);
+    s.shrink_to_fit();
+    ASSERT_EQ(0, memcmp(s.data(), random_bytes.get(), new_size));
+    ASSERT_EQ(std::max<int>(faststring::kInitialCapacity, new_size), s.capacity());
+  }
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/dce25916/src/kudu/util/faststring.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/faststring.cc b/src/kudu/util/faststring.cc
index cf5dbd1..a1cd26b 100644
--- a/src/kudu/util/faststring.cc
+++ b/src/kudu/util/faststring.cc
@@ -18,8 +18,7 @@
 #include "kudu/util/faststring.h"
 
 #include <glog/logging.h>
-
-#include "kudu/gutil/gscoped_ptr.h"
+#include <memory>
 
 namespace kudu {
 
@@ -38,7 +37,7 @@ void faststring::GrowByAtLeast(size_t count) {
 
 void faststring::GrowArray(size_t newcapacity) {
   DCHECK_GE(newcapacity, capacity_);
-  gscoped_array<uint8_t> newdata(new uint8_t[newcapacity]);
+  std::unique_ptr<uint8_t[]> newdata(new uint8_t[newcapacity]);
   if (len_ > 0) {
     memcpy(&newdata[0], &data_[0], len_);
   }
@@ -53,5 +52,21 @@ void faststring::GrowArray(size_t newcapacity) {
   ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
 }
 
+void faststring::ShrinkToFitInternal() {
+  DCHECK_NE(data_, initial_data_);
+  if (len_ <= kInitialCapacity) {
+    ASAN_UNPOISON_MEMORY_REGION(initial_data_, len_);
+    memcpy(initial_data_, &data_[0], len_);
+    delete[] data_;
+    data_ = initial_data_;
+    capacity_ = kInitialCapacity;
+  } else {
+    std::unique_ptr<uint8_t[]> newdata(new uint8_t[len_]);
+    memcpy(&newdata[0], &data_[0], len_);
+    delete[] data_;
+    data_ = newdata.release();
+    capacity_ = len_;
+  }
+}
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/dce25916/src/kudu/util/faststring.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/faststring.h b/src/kudu/util/faststring.h
index 5bd003a..3d25c84 100644
--- a/src/kudu/util/faststring.h
+++ b/src/kudu/util/faststring.h
@@ -30,6 +30,10 @@ namespace kudu {
 // instead of memsetting to \0)
 class faststring {
  public:
+  enum {
+    kInitialCapacity = 32
+  };
+
   faststring() :
     data_(initial_data_),
     len_(0),
@@ -198,6 +202,18 @@ class faststring {
                 str.size());
   }
 
+  // Reallocates the internal storage to fit only the current data.
+  //
+  // This may revert to using internal storage if the current length is shorter than
+  // kInitialCapacity. Note that, in that case, after this call, capacity() will return
+  // a capacity larger than the data length.
+  //
+  // Any pointers within this instance are invalidated.
+  void shrink_to_fit() {
+    if (data_ == initial_data_ || capacity_ == len_) return;
+    ShrinkToFitInternal();
+  }
+
   // Return a copy of this string as a std::string.
   std::string ToString() const {
     return std::string(reinterpret_cast<const char *>(data()),
@@ -227,9 +243,7 @@ class faststring {
   // the current capacity.
   void GrowArray(size_t newcapacity);
 
-  enum {
-    kInitialCapacity = 32
-  };
+  void ShrinkToFitInternal();
 
   uint8_t* data_;
   uint8_t initial_data_[kInitialCapacity];


[2/4] kudu git commit: log: fix some incorrect assumptions on BlockingQueue shutdown

Posted by al...@apache.org.
log: fix some incorrect assumptions on BlockingQueue shutdown

There was an incorrect comment in the Log::AppendThread body in which it
claimed that BlockingQueue::Drain would potentially return false and
also return elements at the same time.

This was in fact not the case. This patch adds unit tests to verify
the actual behavior, and simplifies the code accordingly.

Change-Id: I43e1304ce242f2def26a29504b76bb3e74a93ca3
Reviewed-on: http://gerrit.cloudera.org:8080/6855
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/43fbfdff
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/43fbfdff
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/43fbfdff

Branch: refs/heads/master
Commit: 43fbfdff9abc1516c8489e2f477e8b9fca22878f
Parents: e6141a0
Author: Todd Lipcon <to...@cloudera.com>
Authored: Wed May 10 21:11:34 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu May 11 06:01:15 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log.cc            | 11 +++--------
 src/kudu/util/blocking_queue-test.cc | 25 +++++++++++++++++++++++++
 src/kudu/util/blocking_queue.h       |  9 +++++++--
 3 files changed, 35 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/43fbfdff/src/kudu/consensus/log.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 4a6121d..50ce12b 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -193,18 +193,13 @@ Status Log::AppendThread::Init() {
 }
 
 void Log::AppendThread::RunThread() {
-  bool shutting_down = false;
-  while (PREDICT_TRUE(!shutting_down)) {
+  while (true) {
     vector<LogEntryBatch*> entry_batches;
     ElementDeleter d(&entry_batches);
 
-    // We shut down the entry_queue when it's time to shut down the append
-    // thread, which causes this call to return false, while still populating
-    // the entry_batches vector with the final set of log entry batches that
-    // were enqueued. We finish processing this last bunch of log entry batches
-    // before exiting the main RunThread() loop.
     if (PREDICT_FALSE(!log_->entry_queue()->BlockingDrainTo(&entry_batches))) {
-      shutting_down = true;
+      CHECK(entry_batches.empty());
+      break;
     }
 
     if (log_->metrics_) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/43fbfdff/src/kudu/util/blocking_queue-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/blocking_queue-test.cc b/src/kudu/util/blocking_queue-test.cc
index cce21bf..917ada5 100644
--- a/src/kudu/util/blocking_queue-test.cc
+++ b/src/kudu/util/blocking_queue-test.cc
@@ -65,6 +65,31 @@ TEST(BlockingQueueTest, TestBlockingDrainTo) {
   ASSERT_EQ(3, out[2]);
 }
 
+// Test that, when the queue is shut down with elements still pending,
+// Drain still returns true until the elements are all gone.
+TEST(BlockingQueueTest, TestGetAndDrainAfterShutdown) {
+  // Put some elements into the queue and then shut it down.
+  BlockingQueue<int32_t> q(3);
+  ASSERT_EQ(q.Put(1), QUEUE_SUCCESS);
+  ASSERT_EQ(q.Put(2), QUEUE_SUCCESS);
+
+  q.Shutdown();
+
+  // Get() should still return an element.
+  int i;
+  ASSERT_TRUE(q.BlockingGet(&i));
+  ASSERT_EQ(1, i);
+
+  // Drain should still return true, since it yielded elements.
+  vector<int32_t> out;
+  ASSERT_TRUE(q.BlockingDrainTo(&out));
+  ASSERT_EQ(2, out[0]);
+
+  // Now that it's empty, it should return false.
+  ASSERT_FALSE(q.BlockingDrainTo(&out));
+  ASSERT_FALSE(q.BlockingGet(&i));
+}
+
 TEST(BlockingQueueTest, TestTooManyInsertions) {
   BlockingQueue<int32_t> test_queue(2);
   ASSERT_EQ(test_queue.Put(123), QUEUE_SUCCESS);

http://git-wip-us.apache.org/repos/asf/kudu/blob/43fbfdff/src/kudu/util/blocking_queue.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/blocking_queue.h b/src/kudu/util/blocking_queue.h
index b44c129..86a4c48 100644
--- a/src/kudu/util/blocking_queue.h
+++ b/src/kudu/util/blocking_queue.h
@@ -99,8 +99,13 @@ class BlockingQueue {
     return true;
   }
 
-  // Get all elements from the queue and append them to a
-  // vector. Returns false if shutdown prior to getting the elements.
+  // Get all elements from the queue and append them to a vector.
+  //
+  // If the queue has been shut down, but there are still elements waiting,
+  // then it returns those elements as if the queue were not yet shut down.
+  //
+  // Returns false if the queue has been shut down and has no more remaining
+  // elements.
   bool BlockingDrainTo(std::vector<T>* out) {
     MutexLock l(lock_);
     while (true) {


[4/4] kudu git commit: [catalog_manager_tsk-itest] shortened test's run-time

Posted by al...@apache.org.
[catalog_manager_tsk-itest] shortened test's run-time

The longer the test runs the more tables it creates and drops,
and that makes switching from one leader master to another longer.
The presence of some parallel activity exacerbates the problem because
re-elections happen more often.  That leads to timeouts on operations
performed by the client.

This patch shortens the test's run time to make it less flaky if running
on inferior VMs with other concurrent activity.

Change-Id: I21cca3d53d033f56f60e46fb5a3f67f5d7e44122
Reviewed-on: http://gerrit.cloudera.org:8080/6854
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <da...@gmail.com>


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

Branch: refs/heads/master
Commit: de1daf282f1d045ab2fe3652771f545e67551367
Parents: dce2591
Author: Alexey Serbin <as...@cloudera.com>
Authored: Wed May 10 21:45:28 2017 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu May 11 18:12:08 2017 +0000

----------------------------------------------------------------------
 .../integration-tests/catalog_manager_tsk-itest.cc     | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/de1daf28/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
index 60bc55d..656d15a 100644
--- a/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
+++ b/src/kudu/integration-tests/catalog_manager_tsk-itest.cc
@@ -52,11 +52,12 @@ class CatalogManagerTskITest : public KuduTest {
         num_tservers_(1),
 #if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER)
         hb_interval_ms_(32),
+        run_time_seconds_(5)
 #else
         hb_interval_ms_(16),
+        run_time_seconds_(AllowSlowTests() ? 100 : 5)
 #endif
-        run_time_seconds_(AllowSlowTests() ? 300 : 10) {
-
+        {
     cluster_opts_.num_masters = num_masters_;
     cluster_opts_.master_rpc_ports = { 11030, 11031, 11032 };
     cluster_opts_.num_tablet_servers = num_tservers_;
@@ -64,6 +65,11 @@ class CatalogManagerTskITest : public KuduTest {
     // Add common flags for both masters and tservers.
     const vector<string> common_flags = {
       Substitute("--raft_heartbeat_interval_ms=$0", hb_interval_ms_),
+      // Added a workaround for memory accounting bug, otherwise the assertion
+      // from MemTracker destructor fires in rare cases. Since the memory
+      // accounting code is slated for revamping, let's disable the memory
+      // tracking for this test.
+      "--tablet_transaction_memory_limit_mb=-1",
     };
     copy(common_flags.begin(), common_flags.end(),
         back_inserter(cluster_opts_.extra_master_flags));
@@ -79,7 +85,6 @@ class CatalogManagerTskITest : public KuduTest {
       "--leader_failure_max_missed_heartbeat_periods=1.0",
       "--master_non_leader_masters_propagate_tsk",
       "--tsk_rotation_seconds=2",
-      Substitute("--authn_token_validity_seconds=$0", run_time_seconds_),
     };
     copy(master_flags.begin(), master_flags.end(),
         back_inserter(cluster_opts_.extra_master_flags));
@@ -101,7 +106,7 @@ class CatalogManagerTskITest : public KuduTest {
     using ::kudu::client::sp::shared_ptr;
     static const char* kTableName = "test-table";
     // Using the setting for both RPC and admin operation timeout.
-    const MonoDelta timeout = MonoDelta::FromSeconds(run_time_seconds_);
+    const MonoDelta timeout = MonoDelta::FromSeconds(600);
     KuduClientBuilder builder;
     builder.default_admin_operation_timeout(timeout).default_rpc_timeout(timeout);
     shared_ptr<KuduClient> client;