You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/03/06 19:54:24 UTC

[1/2] kudu git commit: KUDU-1906. Fix lost callback for scanner path

Repository: kudu
Updated Branches:
  refs/heads/branch-1.3.x 6139cdd69 -> c8cbc8b5c


KUDU-1906. Fix lost callback for scanner path

Fixes another case similar to KUDU-1888 in which we were sending an RPC
before setting its deferred. In the case that the RPC responded very
quickly, the response would come before the callback was attached, and
the callback would never get called.

This caused my RowCounter jobs on a small/underpowered test cluster to
have task timeouts a few percent of the time.

This patch fixes the particular instance and also adds some assertions
to try to prevent this style of bug in the future.

Change-Id: I102778e87d0f153cdd2a1ca2aed3ec1e17014d4b
Reviewed-on: http://gerrit.cloudera.org:8080/6239
Tested-by: Kudu Jenkins
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>
(cherry picked from commit 7f2624ae4e0132aee24f6b7b2af31e2219ac31f0)
Reviewed-on: http://gerrit.cloudera.org:8080/6253
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/branch-1.3.x
Commit: 488fc79f1563f32ed9ffe686f6e6a6e35dbf61b6
Parents: 6139cdd
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Mar 2 19:13:57 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Mar 3 22:48:45 2017 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/kudu/client/AsyncKuduClient.java   | 3 ++-
 .../src/main/java/org/apache/kudu/client/KuduRpc.java           | 5 +++++
 .../src/main/java/org/apache/kudu/client/TabletClient.java      | 1 +
 .../test/java/org/apache/kudu/client/TestAsyncKuduSession.java  | 3 ++-
 4 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/488fc79f/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index 6639284..f1c8adc 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -683,8 +683,9 @@ public class AsyncKuduClient implements AutoCloseable {
           " will retry after a delay");
       return delayedSendRpcToTablet(nextRequest, new RecoverableException(statusRemoteError));
     }
+    Deferred<AsyncKuduScanner.Response> d = nextRequest.getDeferred();
     client.sendRpc(nextRequest);
-    return nextRequest.getDeferred();
+    return d;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kudu/blob/488fc79f/java/kudu-client/src/main/java/org/apache/kudu/client/KuduRpc.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduRpc.java b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduRpc.java
index 4a8b971..09105fc 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduRpc.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduRpc.java
@@ -223,6 +223,7 @@ public abstract class KuduRpc<R> {
   private void handleCallback(final Object result) {
     final Deferred<R> d = deferred;
     if (d == null) {
+      LOG.debug("Handling a callback on RPC {} with no deferred attached!", this);
       return;
     }
     deferred = null;
@@ -299,6 +300,10 @@ public abstract class KuduRpc<R> {
     return deferred;
   }
 
+  boolean hasDeferred() {
+    return deferred != null;
+  }
+
   RemoteTablet getTablet() {
     return this.tablet;
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/488fc79f/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
index 938af00..4d44406 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
@@ -152,6 +152,7 @@ public class TabletClient extends SimpleChannelUpstreamHandler {
   }
 
   <R> void sendRpc(KuduRpc<R> rpc) {
+    Preconditions.checkArgument(rpc.hasDeferred());
     rpc.addTrace(
         new RpcTraceFrame.RpcTraceFrameBuilder(
             rpc.method(),

http://git-wip-us.apache.org/repos/asf/kudu/blob/488fc79f/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
index b7f9db3..d84f972 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
@@ -150,8 +150,9 @@ public class TestAsyncKuduSession extends BaseKuduTest {
       // Wait until tablet is deleted on TS.
       while (true) {
         ListTabletsRequest req = new ListTabletsRequest();
+        Deferred<ListTabletsResponse> d = req.getDeferred();
         tc.sendRpc(req);
-        ListTabletsResponse resp = req.getDeferred().join();
+        ListTabletsResponse resp = d.join();
         if (!resp.getTabletsList().contains(tabletId)) {
           break;
         }


[2/2] kudu git commit: KUDU-1905 - Allow reinserts on pk only tables

Posted by mp...@apache.org.
KUDU-1905 - Allow reinserts on pk only tables

Doing a reinsert to a table that has only primary key columns
results in an empty change list. We're currently crashing whenever
we see a empty changelist that is not a delete.

The fix is just to allow empty changelists for reinserts.
This also adds a new flavor of fuzz tests to fuzz-itest.cc
that only have pk-only operations, as well as a directed
regression test that would trigger the problem deterministically.

Ran fuzz-itest in dist-tests with the new tests and the following
command:
KUDU_ALLOW_SLOW_TESTS=1 build-support/dist_test.py --collect-tmpdir \
loop -n 1000 build/debug/bin/fuzz-itest --gtest_repeat=10 \
--gtest_break_on_failure

Tests passed 1000/1000. Results:
http://dist-test.cloudera.org//job?job_id=david.alves.1488580839.22665

Change-Id: I7ce03378c7b97fac8ad8cb7783dec4a1b0277344
Reviewed-on: http://gerrit.cloudera.org:8080/6258
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
(cherry picked from commit fff5cbda80b6cc2016c4bce012d2e183d3c3e2bb)
Reviewed-on: http://gerrit.cloudera.org:8080/6269


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

Branch: refs/heads/branch-1.3.x
Commit: c8cbc8b5c6be31604ba8838e7ac138083516ce31
Parents: 488fc79
Author: David Alves <dr...@apache.org>
Authored: Fri Mar 3 14:17:58 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Mar 6 19:37:58 2017 +0000

----------------------------------------------------------------------
 src/kudu/common/row_changelist-test.cc   |   6 --
 src/kudu/common/row_changelist.cc        |   6 +-
 src/kudu/integration-tests/fuzz-itest.cc | 137 +++++++++++++++++++++-----
 3 files changed, 118 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c8cbc8b5/src/kudu/common/row_changelist-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_changelist-test.cc b/src/kudu/common/row_changelist-test.cc
index 9523322..6b934a2 100644
--- a/src/kudu/common/row_changelist-test.cc
+++ b/src/kudu/common/row_changelist-test.cc
@@ -199,12 +199,6 @@ TEST_F(TestRowChangeList, TestInvalid_TooLongDelete) {
                       "Corruption: DELETE changelist too long");
 }
 
-TEST_F(TestRowChangeList, TestInvalid_TooShortReinsert) {
-  RowChangeListDecoder decoder(RowChangeList(Slice("\x03")));
-  ASSERT_STR_CONTAINS(decoder.Init().ToString(),
-                      "Corruption: empty changelist - expected column updates");
-}
-
 TEST_F(TestRowChangeList, TestInvalid_SetNullForNonNullableColumn) {
   faststring buf;
   RowChangeListEncoder rcl(&buf);

http://git-wip-us.apache.org/repos/asf/kudu/blob/c8cbc8b5/src/kudu/common/row_changelist.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_changelist.cc b/src/kudu/common/row_changelist.cc
index 82b6cc3..cd73d46 100644
--- a/src/kudu/common/row_changelist.cc
+++ b/src/kudu/common/row_changelist.cc
@@ -158,9 +158,11 @@ Status RowChangeListDecoder::Init() {
 
   remaining_.remove_prefix(1);
 
-  // We should discard empty REINSERT/UPDATE RowChangeLists, so if after getting
+  // We should discard empty UPDATE RowChangeLists, so if after getting
   // the type remaining_ is empty() return an error.
-  if (!is_delete() && remaining_.empty()) {
+  // Note that REINSERTs might have empty changelists when reinserting a row on a tablet that
+  // has only primary key columns.
+  if (is_update() && remaining_.empty()) {
     return Status::Corruption("empty changelist - expected column updates");
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/c8cbc8b5/src/kudu/integration-tests/fuzz-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/fuzz-itest.cc b/src/kudu/integration-tests/fuzz-itest.cc
index b88148d..e98afb5 100644
--- a/src/kudu/integration-tests/fuzz-itest.cc
+++ b/src/kudu/integration-tests/fuzz-itest.cc
@@ -53,6 +53,7 @@ DECLARE_bool(use_hybrid_clock);
 // the fuzz test.
 enum TestOpType {
   TEST_INSERT,
+  TEST_INSERT_PK_ONLY,
   TEST_UPSERT,
   TEST_UPSERT_PK_ONLY,
   TEST_UPDATE,
@@ -101,6 +102,7 @@ namespace tablet {
 
 const char* TestOpType_names[] = {
   "TEST_INSERT",
+  "TEST_INSERT_PK_ONLY",
   "TEST_UPSERT",
   "TEST_UPSERT_PK_ONLY",
   "TEST_UPDATE",
@@ -130,6 +132,33 @@ struct TestOp {
   }
 };
 
+const vector<TestOpType> kAllOps {TEST_INSERT,
+                                  TEST_INSERT_PK_ONLY,
+                                  TEST_UPSERT,
+                                  TEST_UPSERT_PK_ONLY,
+                                  TEST_UPDATE,
+                                  TEST_DELETE,
+                                  TEST_FLUSH_OPS,
+                                  TEST_FLUSH_TABLET,
+                                  TEST_FLUSH_DELTAS,
+                                  TEST_MINOR_COMPACT_DELTAS,
+                                  TEST_MAJOR_COMPACT_DELTAS,
+                                  TEST_COMPACT_TABLET,
+                                  TEST_RESTART_TS,
+                                  TEST_SCAN_AT_TIMESTAMP};
+
+const vector<TestOpType> kPkOnlyOps {TEST_INSERT_PK_ONLY,
+                                     TEST_UPSERT_PK_ONLY,
+                                     TEST_DELETE,
+                                     TEST_FLUSH_OPS,
+                                     TEST_FLUSH_TABLET,
+                                     TEST_FLUSH_DELTAS,
+                                     TEST_MINOR_COMPACT_DELTAS,
+                                     TEST_MAJOR_COMPACT_DELTAS,
+                                     TEST_COMPACT_TABLET,
+                                     TEST_RESTART_TS,
+                                     TEST_SCAN_AT_TIMESTAMP};
+
 // Test which does only random operations against a tablet, including update and random
 // get (ie scans with equal lower and upper bounds).
 //
@@ -142,11 +171,10 @@ class FuzzTest : public KuduTest {
     FLAGS_enable_maintenance_manager = false;
     FLAGS_use_hybrid_clock = false;
     FLAGS_scanner_allow_snapshot_scans_with_logical_timestamps = true;
-
-    schema_ = client::KuduSchemaFromSchema(CreateKeyValueTestSchema());
   }
 
-  void SetUp() override {
+  void CreateTabletAndStartClusterWithSchema(const Schema& schema) {
+    schema_ =  client::KuduSchemaFromSchema(schema);
     KuduTest::SetUp();
 
     MiniClusterOptions opts;
@@ -175,8 +203,8 @@ class FuzzTest : public KuduTest {
   }
 
   void TearDown() override {
-    tablet_peer_.reset();
-    cluster_->Shutdown();
+    if (tablet_peer_) tablet_peer_.reset();
+    if (cluster_) cluster_->Shutdown();
   }
 
   scoped_refptr<TabletPeer> LookupTabletPeer() {
@@ -211,7 +239,7 @@ class FuzzTest : public KuduTest {
                                         TestOpType type) {
     ExpectedKeyValueRow ret;
     unique_ptr<KuduWriteOperation> op;
-    if (type == TEST_INSERT) {
+    if (type == TEST_INSERT || type == TEST_INSERT_PK_ONLY) {
       op.reset(table_->NewInsert());
     } else {
       op.reset(table_->NewUpsert());
@@ -219,17 +247,26 @@ class FuzzTest : public KuduTest {
     KuduPartialRow* row = op->mutable_row();
     CHECK_OK(row->SetInt32(0, key));
     ret.key = key;
-    if (type != TEST_UPSERT_PK_ONLY) {
-      if (val & 1) {
-        CHECK_OK(row->SetNull(1));
-      } else {
-        CHECK_OK(row->SetInt32(1, val));
-        ret.val = val;
+    switch (type) {
+      case TEST_INSERT:
+      case TEST_UPSERT: {
+        if (val & 1) {
+          CHECK_OK(row->SetNull(1));
+        } else {
+          CHECK_OK(row->SetInt32(1, val));
+          ret.val = val;
+        }
+        break;
       }
-    } else {
-      // For "upsert PK only", we expect the row to keep its old value
-      // the row existed, or NULL if there was no old row.
-      ret.val = old_row ? old_row->val : boost::none;
+      case TEST_INSERT_PK_ONLY:
+        break;
+      case TEST_UPSERT_PK_ONLY: {
+        // For "upsert PK only", we expect the row to keep its old value if
+        // the row existed, or NULL if there was no old row.
+        ret.val = old_row ? old_row->val : boost::none;
+        break;
+      }
+      default: LOG(FATAL) << "Invalid test op type: " << TestOpType_names[type];
     }
     CHECK_OK(session_->Apply(op.release()));
     return ret;
@@ -276,7 +313,7 @@ class FuzzTest : public KuduTest {
       for (KuduScanBatch::RowPtr row : batch) {
         ExpectedKeyValueRow ret;
         CHECK_OK(row.GetInt32(0, &ret.key));
-        if (!row.IsNull(1)) {
+        if (schema_.num_columns() > 1 && !row.IsNull(1)) {
           ret.val = 0;
           CHECK_OK(row.GetInt32(1, ret.val.get_ptr()));
         }
@@ -356,7 +393,7 @@ class FuzzTest : public KuduTest {
       for (KuduScanBatch::RowPtr row : batch) {
         ExpectedKeyValueRow ret;
         ASSERT_OK(row.GetInt32(0, &ret.key));
-        if (!row.IsNull(1)) {
+        if (schema_.num_columns() > 1 && !row.IsNull(1)) {
           ret.val = 0;
           ASSERT_OK(row.GetInt32(1, ret.val.get_ptr()));
         }
@@ -393,9 +430,25 @@ class FuzzTest : public KuduTest {
   scoped_refptr<TabletPeer> tablet_peer_;
 };
 
+// The set of ops to draw from.
+enum TestOpSets {
+  ALL, // Pick an operation at random from all possible operations.
+  PK_ONLY // Pick an operation at random from the set of operations that apply only to the
+          // primary key (or that are now row-specific, like flushes or compactions).
+};
+
+TestOpType PickOpAtRandom(TestOpSets sets) {
+  switch (sets) {
+    case ALL:
+      return kAllOps[rand() % kAllOps.size()];
+    case PK_ONLY:
+      return kPkOnlyOps[rand() % kPkOnlyOps.size()];
+  }
+}
+
 // Generate a random valid sequence of operations for use as a
 // fuzz test.
-void GenerateTestCase(vector<TestOp>* ops, int len) {
+void GenerateTestCase(vector<TestOp>* ops, int len, TestOpSets sets = ALL) {
   vector<bool> exists(FLAGS_keyspace_size);
   int op_timestamps = 0;
   bool ops_pending = false;
@@ -404,12 +457,13 @@ void GenerateTestCase(vector<TestOp>* ops, int len) {
   bool data_in_dms = false;
   ops->clear();
   while (ops->size() < len) {
-    TestOpType r = tight_enum_cast<TestOpType>(rand() % enum_limits<TestOpType>::max_enumerator);
+    TestOpType r = PickOpAtRandom(sets);
     int row_key = rand() % FLAGS_keyspace_size;
     switch (r) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
         if (exists[row_key]) continue;
-        ops->push_back({TEST_INSERT, row_key});
+        ops->push_back({r, row_key});
         exists[row_key] = true;
         ops_pending = true;
         data_in_mrs = true;
@@ -502,7 +556,7 @@ void GenerateTestCase(vector<TestOp>* ops, int len) {
         break;
       }
       default:
-        LOG(FATAL);
+        LOG(FATAL) << "Invalid op type: " << r;
     }
   }
 }
@@ -543,6 +597,7 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
   for (const TestOp& test_op : test_ops) {
     switch (test_op.type) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
       case TEST_UPSERT:
       case TEST_UPSERT_PK_ONLY:
       case TEST_UPDATE:
@@ -555,6 +610,7 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
     LOG(INFO) << test_op.ToString();
     switch (test_op.type) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
       case TEST_UPSERT:
       case TEST_UPSERT_PK_ONLY: {
         pending_val[test_op.val] = InsertOrUpsertRow(
@@ -612,7 +668,19 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
 // Generates a random test sequence and runs it.
 // The logs of this test are designed to easily be copy-pasted and create
 // more specific test cases like TestFuzz<N> below.
+TEST_F(FuzzTest, TestRandomFuzzPksOnly) {
+  CreateTabletAndStartClusterWithSchema(Schema({ColumnSchema("key", INT32)}, 1));
+  SeedRandom();
+  vector<TestOp> test_ops;
+  GenerateTestCase(&test_ops, AllowSlowTests() ? 1000 : 50, TestOpSets::PK_ONLY);
+  RunFuzzCase(test_ops);
+}
+
+// Generates a random test sequence and runs it.
+// The logs of this test are designed to easily be copy-pasted and create
+// more specific test cases like TestFuzz<N> below.
 TEST_F(FuzzTest, TestRandomFuzz) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   SeedRandom();
   vector<TestOp> test_ops;
   GenerateTestCase(&test_ops, AllowSlowTests() ? 1000 : 50);
@@ -623,6 +691,7 @@ TEST_F(FuzzTest, TestRandomFuzz) {
 // This results in very large batches which are likely to span multiple delta blocks
 // when flushed.
 TEST_F(FuzzTest, TestRandomFuzzHugeBatches) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   SeedRandom();
   vector<TestOp> test_ops;
   GenerateTestCase(&test_ops, AllowSlowTests() ? 500 : 50);
@@ -637,6 +706,7 @@ TEST_F(FuzzTest, TestRandomFuzzHugeBatches) {
 }
 
 TEST_F(FuzzTest, TestFuzz1) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
       // Get an inserted row in a DRS.
       {TEST_INSERT, 0},
@@ -662,6 +732,7 @@ TEST_F(FuzzTest, TestFuzz1) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz2) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_DELETE, 0},
@@ -694,6 +765,7 @@ TEST_F(FuzzTest, TestFuzz2) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz3) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_FLUSH_OPS, 0},
@@ -728,6 +800,7 @@ TEST_F(FuzzTest, TestFuzz3) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz4) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_FLUSH_OPS, 0},
@@ -763,6 +836,7 @@ TEST_F(FuzzTest, TestFuzz4) {
 //  Actual: "()"
 //  Expected: "(" + cur_val + ")"
 TEST_F(FuzzTest, TestFuzzWithRestarts1) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_FLUSH_OPS, 0},
@@ -789,6 +863,7 @@ TEST_F(FuzzTest, TestFuzzWithRestarts1) {
 // insert undo deltas in sorted order (ascending key, then descending ts):
 // got key (row 1@tx5965182714017464320) after (row 1@tx5965182713875046400)
 TEST_F(FuzzTest, TestFuzzWithRestarts2) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 0},
       {TEST_FLUSH_OPS, 0},
@@ -823,6 +898,7 @@ TEST_F(FuzzTest, TestFuzzWithRestarts2) {
 // Regression test for KUDU-1467: a sequence involving
 // UPSERT which failed to replay properly upon bootstrap.
 TEST_F(FuzzTest, TestUpsertSeq) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_UPSERT, 1},
@@ -841,7 +917,8 @@ TEST_F(FuzzTest, TestUpsertSeq) {
 
 // Regression test for KUDU-1623: updates without primary key
 // columns specified can cause crashes and issues at restart.
-TEST_F(FuzzTest, TestUpsert_PKOnly) {
+TEST_F(FuzzTest, TestUpsert_PKOnlyOps) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_FLUSH_OPS, 0},
@@ -851,5 +928,19 @@ TEST_F(FuzzTest, TestUpsert_PKOnly) {
     });
 }
 
+// Regression test for KUDU-1905: reinserts to a tablet that has
+// only primary keys end up as empty change lists. We were previously
+// crashing when a changelist was empty.
+TEST_F(FuzzTest, TestUpsert_PKOnlySchema) {
+  CreateTabletAndStartClusterWithSchema(Schema({ColumnSchema("key", INT32)}, 1));
+  RunFuzzCase({
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_DELETE, 1},
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_FLUSH_OPS, 0}
+     });
+}
+
 } // namespace tablet
 } // namespace kudu