You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Hao Hao (Code Review)" <ge...@cloudera.org> on 2021/02/20 22:42:55 UTC

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Hao Hao has uploaded this change for review. ( http://gerrit.cloudera.org:8080/17097


Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coase grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction at a time, the same transaction
can hold the lock for multiple times. To prevent deadlock, 'wait-die'
scheme is used, which if the transaction requires a lock held by another
transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 281 insertions(+), 7 deletions(-)



  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/1
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 1
Gerrit-Owner: Hao Hao <ha...@cloudera.com>

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#8).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 464 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/8
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 8
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has uploaded a new patch set (#12) to the change originally created by Hao Hao. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock for multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 517 insertions(+), 16 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/12
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 12
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 5:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc@261
PS5, Line 261: partition_lock.IsAcquired(&code)
> Using an already 'moved' variable doesn't smell good.  Is it enough just to
Ah, I guess you can use the moved instances, but maybe add specific NOLINT to be explicit about the intention here and suppress TidyBot warnings


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@409
PS5, Line 409:   manager_ = other->manager_;
nit: add a DCHECK() on other != this, just to catch programming errors.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Thu, 11 Mar 2021 20:22:14 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 13:

Reverted back to using a semaphore, with a slight modifications. Sleeping in the prepare thread did not sit well with me.


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 13
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Thu, 18 Mar 2021 22:15:22 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 11:

Updated this to adjust some language and naming.

Also realized that using a semaphore made it a bit more confusing to follow and reason about re: concurrency, so I reverted to the spinlock implementation, but avoiding the sleep issue mentioned in discussion.

Also added a test case specific to test deadlock protection.


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 11
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Mon, 15 Mar 2021 22:32:06 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 5:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc@261
PS5, Line 261: partition_lock.IsAcquired(&code)
Using an already 'moved' variable doesn't smell good.  Is it enough just to check for the reference count to make sure the former lock isn't held anymore?


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@437
PS5, Line 437:   if (!txn_id.IsValid()) {
             :     id = std::numeric_limits<int64_t>::max();
             :   } else {
             :     id = txn_id.value();
             :   }
nit: this might be done using tri-state operator:

  int64_t id = txn_id.IsValid() ? txn_id.value() : ...;


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@473
PS5, Line 473: PartitionLock* LockManager::WaitUntilAcquiredPartitionLock(const TxnId& txn_id) {
What happens when this is running for too long, going over the overall timeout for an operation?  In other words, should we add a deadline parameter for this method?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Thu, 11 Mar 2021 20:17:28 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@158
PS6, Line 158:   ASSERT_EQ(1, lock_manager_.partition_lock_refs());
> Looking at this as a block box, does it make sense to check that the first 
Done


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@252
PS6, Line 252: if (acquired) {
> Why it might not be acquired?  Is it possible?
This might happen if the non-transactional op take the op first.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@267
PS6, Line 267:     if (acquired) {
             :       CHECK_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :       CHECK_EQ(1, lock_manager_.partition_lock_refs());
             :     }
> I'm not sure I understand this piece if reading the comment for the thread'
Sorry forget to update the comment.. Updated it.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h@193
PS6, Line 193: ScopedPartitionLock
> What if an instance of this class is copied?  Would it result in incorrect 
I don't think so, since we only update the reference at the constructor. But I disabled the copy constructor anyway.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@448
PS6, Line 448: kMaxBackoffExp
> nit: this might be a constexpr
Ack


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@496
PS6, Line 496: ms
> Is it in seconds or milliseconds?
Ah, sorry for the confusion. It is in milliseconds, since I think we don't need to wait at the second level, as the caller() should already know the lock can be acquired (fast) with TryAcquirePartitionLock() succeeded previously. WDYT?

I moved the log to TryAcquirePartitionLock() to avoid the report being screwed.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@520
PS6, Line 520:   partition_lock_refs_ -= 1;
             :   DCHECK_GE(partition_lock_refs_, 0);
             :   if (partition_lock_refs_ == 0) {
             :     partition_lock_.reset();
             :   }
> nit: this might be shortened to
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 06:28:23 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has uploaded a new patch set (#15) to the change originally created by Hao Hao. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse-grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 540 insertions(+), 16 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/15
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 15
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#5).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 396 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/5
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 3: Code-Review+1

(3 comments)

http://gerrit.cloudera.org:8080/#/c/17097/3//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/17097/3//COMMIT_MSG@12
PS3, Line 12: hold
nit: acquire?


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc@195
PS3, Line 195:     threads.emplace_back([&] {
> warning: 'emplace_back' is called inside a loop; consider pre-allocating th
nit: since lock acquisition is quite fast operation, maybe it's worth synchronizing the start of the threads' activity by a barrier?


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc@208
PS3, Line 208: TEST_F(LockManagerTest, TestConcurrentLockUnlockPartitionWithInvalidID) {
nit: since lock acquisition is quite fast operation, maybe it's worth synchronizing the start of the threads' activity by a barrier?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 3
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 26 Feb 2021 18:07:44 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 10:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h@193
PS6, Line 193: 
> Yeah, we updated the reference in the constructor, so if copying field-to-f
Ah makes sense. Sorry I missed that! Thanks a lot for pointing out.


http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc@460
PS9, Line 460: (MonoDelta::FromMilliseconds(25
> This seems to be pretty short timeout.  Why just a millisecond?  Our timeou
Ack


http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc@474
PS9, Line 474:      if (!partition_lock_) {
> What if by this time the lock has been released by the other party?  Will i
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 10
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 21:33:57 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 8:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager-test.cc@214
PS8, Line 214: ] {
We need to pass 't' as a copy.


http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager.cc@461
PS8, Line 461:     MonoTime now(MonoTime::Now());
             :     MonoDelta elapsed(now - start);
nit: just store elapsed and inline MonoTime::Now()?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 8
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 07:54:31 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 5:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

PS5: 
Could you also add some (maybe multithreaded) tests for the Wait... functionality?


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.h@81
PS5, Line 81:   // Similar to the above, but wait until the lock is acquired.
nit: could you mention that the caller is expected to ensure there is no deadlock? E.g. by only running on followers in the prepare phase, or running serially in an order that has already been successful with TryAcquirePartitionLock() calls?


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@157
PS3, Line 157:  multi-rows t
> I found it is more clear to have a specific entity to represent the lock. E
Yeah, if anything maybe we could call this PartitionLockState or something


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@367
PS5, Line 367: bool must_acquire
nit: maybe use an enum?

enum LockWaitMode {
  kWaitForLock,
  kTryLock,
};


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@372
PS5, Line 372: DCHECK_NOTNULL(lock_);
nit: can just used DCHECK(lock_)

DCHECK_NOTNULL() returns the pointer, so this is creates an unused reference


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@478
PS5, Line 478: (lock = TryAcquirePartitionLock(txn_id, &code))
Would it make sense to extend the TryAcquirePartitionLock() method with a timeout, similar to the semaphore at L537? I guess we'll see how frequently this codepath gets hit based on how frequently there are wait-die conflicts..



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Thu, 11 Mar 2021 20:39:56 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@485
PS6, Line 485:     SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
> The point here is that the lock based on spinlock is taken at line 452 and 
Ah I though you are referring to the partition lock. Makes sense, updated it.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 05:03:49 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@485
PS6, Line 485:     SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
It's bad practice to sleep in a thread that is holding a lock. Is there any way we could use a semaphore instead, like we do for the row locks? Perhaps add it to the LockManager as a member?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 00:53:39 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 14: Code-Review+1


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 14
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 15:25:15 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 5:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

PS5: 
> Could you also add some (maybe multithreaded) tests for the Wait... functio
Ah right, missed that.


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager-test.cc@261
PS5, Line 261: partition_lock.IsAcquired(&code)
> Ah, I guess you can use the moved instances, but maybe add specific NOLINT 
Ack


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.h@81
PS5, Line 81:   // Similar to the above, but wait until the lock is acquired.
> nit: could you mention that the caller is expected to ensure there is no de
Done


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@367
PS5, Line 367: bool must_acquire
> nit: maybe use an enum?
Done


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@372
PS5, Line 372: DCHECK_NOTNULL(lock_);
> nit: can just used DCHECK(lock_)
Done


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@409
PS5, Line 409:   manager_ = other->manager_;
> nit: add a DCHECK() on other != this, just to catch programming errors.
Done


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@437
PS5, Line 437:   if (!txn_id.IsValid()) {
             :     id = std::numeric_limits<int64_t>::max();
             :   } else {
             :     id = txn_id.value();
             :   }
> nit: this might be done using tri-state operator:
Done


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@473
PS5, Line 473: PartitionLock* LockManager::WaitUntilAcquiredPartitionLock(const TxnId& txn_id) {
> What happens when this is running for too long, going over the overall time
Yeah, this is used for case where we expect the lock can be acquired eventually (e.g. for a follower to replicate the participanOp).


http://gerrit.cloudera.org:8080/#/c/17097/5/src/kudu/tablet/lock_manager.cc@478
PS5, Line 478: (lock = TryAcquirePartitionLock(txn_id, &code))
> Would it make sense to extend the TryAcquirePartitionLock() method with a t
Makes sense, updated it.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 00:18:21 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 13: Code-Review+1

(6 comments)

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc@267
PS13, Line 267:           }
nit: does it make sense to add an ASSERT to make sure the lock is either acquired or the 'code' is one of two options,  but nothing else?


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc@270
PS13, Line 270:       // Have each transaction attempt to lock one lock manager and then the
Very nice scenario -- thank you for adding this one.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h@193
PS6, Line 193:  lock is released w
> Ah makes sense. Sorry I missed that! Thanks a lot for pointing out.
Not a problem at all: most importantly that's fixed now.  Thank you very much for addressing this!


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@485
PS13, Line 485:       const MonoDelta elapsed(MonoTime::Now() - start);
nit: is it possible to move this under the 'if (timeout.Initialized())' case to avoid making calls to MonoTime::Now() if 'timeout' isn't initialized?


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@521
PS13, Line 521:   PartitionLockState* lock;
              :   lock = TryAcquirePartitionLock(txn_id, &code, MonoDelta());
nit: combine these two lines into a single one?


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@522
PS13, Line 522: MonoDelta()
nit: maybe, change the signature of this method to make this last parameter a parameter by default?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 13
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 06:44:50 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@485
PS6, Line 485:     SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
> BTW, why to hold that spinlock-based lock when sleeping (i.e., why not to r
Hmm, we are only sleeping if we cannot hold the lock. But I will try to use a semaphore instead.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 03:56:59 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#7).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 464 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/7
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 7
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#10).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 470 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/10
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 10
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 2:

(18 comments)

http://gerrit.cloudera.org:8080/#/c/17097/2//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/17097/2//COMMIT_MSG@17
PS2, Line 17: transaction
nit: you meant a write operation here, not the whole transaction, right?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@223
PS2, Line 223: }
nit: are there any post-conditions to check after releasing all the row locks?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@46
PS2, Line 46: // Super-simple lock manager implementation. This only supports exclusive
            : // locks, and makes no attempt to prevent deadlocks if a single thread
            : // takes multiple locks.
            : //
            : // In the future when we want to support multi-row transactions of some kind
            : // we'll have to implement a proper lock manager with all its trappings,
            : // but this should be enough for the single-row use case.
nit: maybe, update this to reflect changes in this patch?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@85
PS2, Line 85:   // Lock to protect 'partition_lock_' and
            :   // 'partition_lock_refs_'.
nit: might be at a single line


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@90
PS2, Line 90: by
            :   // the same transaction.
nit: do you mind extending the comment to explain where the information to identify that 'the same' transaction is stored?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@154
PS2, Line 154: class PartitionLock {
nit: mind adding a short description for this class?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@181
PS2, Line 181: assignment
nit: assignment operator


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@188
PS2, Line 188: Acquired
style nit: rename to 'IsAcquired' or 'is_acquired'


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@189
PS2, Line 189: Release
nit: add a doc?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@365
PS2, Line 365: LockManager::LockMode mode,
I could not find where this parameter is used.  Did I miss anything?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@369
PS2, Line 369: DCHECK_EQ(LockManager::LOCK_EXCLUSIVE, mode);
If this should always be LockManager::LOCK_EXCLUSIVE, why not to drop the 'mode' parameter at all?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@390
PS2, Line 390: manager_->ReleasePartitionLock();
Does 'lock_' needs to be updated after calling ReleasePartitionLock() ?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@392
PS2, Line 392: 
             : ScopedPartitionLock::ScopedPartitionLock(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             : }
             : 
             : ScopedPartitionLock& ScopedPartitionLock::operator=(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             :   return *this;
             : }
> Curious, how do we expect to use these? It isn't clear, but that'd help gau
+1

BTW, what about non-rvalue constructor and assignment operator?  Is it necessary to override them as well?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@415
PS2, Line 415: ()
nit: the parentheses are not required here


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@425
PS2, Line 425: PREDICT_TRUE
Do we expect this to be used by non-transactional operations as well?  If so, is my reading correct that we expect more transactional writes than non-transactional writes?  If so, why?  If not, maybe remove the PREDICT_TRUE hint?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@434
PS2, Line 434:       if (PREDICT_FALSE(txn_id.value() > partition_lock_->txn_id().value())) {
             :         *code = tserver::TabletServerErrorPB::TXN_LOCKED_ABORT;
             :         return nullptr;
             :       }
             :       if (PREDICT_FALSE(txn_id.value() < partition_lock_->txn_id().value())) {
             :         *code = tserver::TabletServerErrorPB::TXN_LOCKED_RETRY;
             :         return nullptr;
             :       }
Looking at this code makes me think that the most anticipated case if when the same transaction tries to acquire the same lock.  Is that so indeed?  If not, maybe remove those PREDICT_FALSE() hints?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@449
PS2, Line 449:   } else {
             :     return nullptr;
             :   }
readability nit: put this condition first and remove the extra 'else' clause


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto
File src/kudu/tserver/tserver.proto:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto@118
PS2, Line 118: TXN_LOCKED_RETRY
nit: maybe, rename into TXN_LOCKED_RETRY_OP to signify that it's an operation, not whole transaction needs to be retried?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 2
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Mon, 22 Feb 2021 17:33:35 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has removed a vote on this change.

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Removed Verified-1 by Kudu Jenkins (120)
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: deleteVote
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 15
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 12:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@157
PS3, Line 157: 
> Yeah, if anything maybe we could call this PartitionLockState or something
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 12
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Mon, 15 Mar 2021 23:57:23 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has uploaded a new patch set (#13) to the change originally created by Hao Hao. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse-grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 534 insertions(+), 16 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/13
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 13
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 9:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager-test.cc@214
PS8, Line 214: , t
> We need to pass 't' as a copy.
Done


http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/8/src/kudu/tablet/lock_manager.cc@461
PS8, Line 461:     MonoDelta elapsed(MonoTime::Now() - start);
             :     if (elapsed > timeout) {
> nit: just store elapsed and inline MonoTime::Now()?
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 9
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 18:14:19 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 9:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h@193
PS6, Line 193: 
> I don't think so, since we only update the reference at the constructor. Bu
Yeah, we updated the reference in the constructor, so if copying field-to-field, then the Release() method is called twice: once from the destructor of the original object, and another time from the field-to-field copied object, and the latter is unexpected extra decrease of the reference count.

Thank you for addressing this!


http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc@460
PS9, Line 460: (MonoDelta::FromMilliseconds(1)
This seems to be pretty short timeout.  Why just a millisecond?  Our timeout aren't strict since this is not a real-time OS anyways.

I'd give it 100ms or 250ms at least -- that way it would be much less useless CPU burn and context switching.


http://gerrit.cloudera.org:8080/#/c/17097/9/src/kudu/tablet/lock_manager.cc@474
PS9, Line 474:      *code = id > partition_lock_->txn_id().value() ?
What if by this time the lock has been released by the other party?  Will it just crash here with SIGSEGV?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 9
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 19:11:21 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 2:

(23 comments)

http://gerrit.cloudera.org:8080/#/c/17097/2//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/17097/2//COMMIT_MSG@17
PS2, Line 17: transaction
> nit: you meant a write operation here, not the whole transaction, right?
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@159
PS2, Line 159: TEST_F(LockManagerTest, TestLockUnlockPartitionMultiTxn) {
             :   // Acquiring a lock that held by another transaction which has a lower txn ID
             :   // will get a 'TXN_LOCKED_ABORT' server error code.
             :   {
             :     TxnId txn1(0);
             :     ScopedPartitionLock first_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn1);
             :     TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
             :     ASSERT_TRUE(first_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :     ASSERT_EQ(1, lock_manager_.partition_lock_refs());
             : 
             :     TxnId txn2(1);
             :     ScopedPartitionLock second_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn2);
             :     ASSERT_FALSE(second_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::TXN_LOCKED_ABORT, code);
             :   }
> nit: probably makes sense to split TXN_LOCKED_ABORT code testing into a sep
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@179
PS2, Line 179:     TxnId txn1(1);
             :     ScopedPartitionLock first_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn1);
             :     TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
             :     ASSERT_TRUE(first_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :     ASSERT_EQ(1, lock_manager_.partition_lock_refs());
             : 
             :     TxnId txn2(0);
> nit: the values of these transaction IDs are pretty confusing IMO. Might be
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@223
PS2, Line 223: }
> nit: are there any post-conditions to check after releasing all the row loc
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@46
PS2, Line 46: // Super-simple lock manager implementation. This only supports exclusive
            : // locks, and makes no attempt to prevent deadlocks if a single thread
            : // takes multiple locks.
            : //
            : // In the future when we want to support multi-row transactions of some kind
            : // we'll have to implement a proper lock manager with all its trappings,
            : // but this should be enough for the single-row use case.
> nit: maybe, update this to reflect changes in this patch?
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@85
PS2, Line 85:   // Lock to protect 'partition_lock_' and
            :   // 'partition_lock_refs_'.
> nit: might be at a single line
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@90
PS2, Line 90: by
            :   // the same transaction.
> nit: do you mind extending the comment to explain where the information to 
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@154
PS2, Line 154: class PartitionLock {
> nit: mind adding a short description for this class?
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@181
PS2, Line 181: assignment
> nit: assignment operator
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@188
PS2, Line 188: Acquired
> style nit: rename to 'IsAcquired' or 'is_acquired'
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.h@189
PS2, Line 189: Release
> nit: add a doc?
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@365
PS2, Line 365: LockManager::LockMode mode,
> I could not find where this parameter is used.  Did I miss anything?
Yes, this is not used. It is actually not used anywhere in the codebase. I added it just to be symmetric with the definition of ScopedRowLock. My understanding is it may be used later when we introduce shared lock. But I can remove it if you think it is redundant.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@368
PS2, Line 368: code_
> Why bother storing this at all if we evaluate it every time we create a Sco
So I think the idea is to be symmetric with the destructor. So that as the class name indicate, as long as the ScopedPartitionLock is in the scope. The lock is held without calling Acquired().


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@369
PS2, Line 369: DCHECK_EQ(LockManager::LOCK_EXCLUSIVE, mode);
> If this should always be LockManager::LOCK_EXCLUSIVE, why not to drop the '
I will remove it.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@390
PS2, Line 390: manager_->ReleasePartitionLock();
> Does 'lock_' needs to be updated after calling ReleasePartitionLock() ?
Hmm, originally I don't think so, as it should not be used it after Release(). However after think it more, it may be more safe to set it to nullptr.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@392
PS2, Line 392: 
             : ScopedPartitionLock::ScopedPartitionLock(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             : }
             : 
             : ScopedPartitionLock& ScopedPartitionLock::operator=(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             :   return *this;
             : }
> +1
These are actually a similar implementation as the ones in of ScopedRowLock. Since I image a similar usage pattern (std::move to transfer the lock) will occur when requiring PartitionLock at each write op. However, as I don't have enough confidence of how this will be used later. So I decided to remove it in this patch. And add it back whenever actually required. Let me know if you think otherwise.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@415
PS2, Line 415: ()
> nit: the parentheses are not required here
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@422
PS2, Line 422: PartitionLock* LockManager::AcquirePartitionLock(
             :     const TxnId& txn_id,
             :     TabletServerErrorPB::Code* code) {
> What happens if a non-transaction attempts to take the lock while a transac
Similar to Alexey's question, at this point I assume the caller is transactional op. So no lock will be returned in this case.

Sure, will do.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@425
PS2, Line 425: PREDICT_TRUE
> Do we expect this to be used by non-transactional operations as well?  If s
Yeah, good point. At this point, I image non-transactional ops will not try to acquire the partition lock as it is not required. However, this may change depends on how the partition lock is checked in a following patch.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@434
PS2, Line 434:       if (PREDICT_FALSE(txn_id.value() > partition_lock_->txn_id().value())) {
             :         *code = tserver::TabletServerErrorPB::TXN_LOCKED_ABORT;
             :         return nullptr;
             :       }
             :       if (PREDICT_FALSE(txn_id.value() < partition_lock_->txn_id().value())) {
             :         *code = tserver::TabletServerErrorPB::TXN_LOCKED_RETRY;
             :         return nullptr;
             :       }
> Looking at this code makes me think that the most anticipated case if when 
Right, since transaction will try to hold the partition lock when the txn begins. If the txn cannot acquire the lock at that moment, it will be aborted or retried. There will be no following write ops which are the majority times when a partition lock is checked.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@449
PS2, Line 449:   } else {
             :     return nullptr;
             :   }
> readability nit: put this condition first and remove the extra 'else' claus
Done


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto
File src/kudu/tserver/tserver.proto:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto@116
PS2, Line 116: transaction
> nit: to be clear, the whole transaction doesn't need to be retried, just th
Right, although my understanding is along with write op, the participant op (Begin TXN) can also be retried if the lock is held by other transactions at that moment.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto@118
PS2, Line 118: TXN_LOCKED_RETRY
> nit: maybe, rename into TXN_LOCKED_RETRY_OP to signify that it's an operati
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 2
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Wed, 24 Feb 2021 01:44:14 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has uploaded a new patch set (#14) to the change originally created by Hao Hao. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse-grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 540 insertions(+), 16 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/14
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 14
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 15: Verified+1

The test failure seems unrelated but worrisome. I filed KUDU-3268 and will look into it soon.


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 15
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 19:31:57 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#9).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 464 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/9
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 9
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse-grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Reviewed-on: http://gerrit.cloudera.org:8080/17097
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-by: Andrew Wong <aw...@cloudera.com>
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 540 insertions(+), 16 deletions(-)

Approvals:
  Alexey Serbin: Looks good to me, approved
  Andrew Wong: Verified

-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 16
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Bankim Bhavsar (Code Review)" <ge...@cloudera.org>.
Bankim Bhavsar has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 3:

(2 comments)

Not a review. Just passing by...

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@19
PS3, Line 19: #include <stdint.h>
Nit: Can use the C++ compatible <cstdint> instead.


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@157
PS3, Line 157: PartitionLock
I find it weird that a class named *Lock doesn't have any lock/unlock/release methods associated with it explicitly or implicitly and doesn't inherit or compose a lock primitive. This might as well be a typedef to TxnId.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 3
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 26 Feb 2021 18:23:07 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has uploaded a new patch set (#11) to the change originally created by Hao Hao. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................

KUDU-2612: add ScopedPartitionLock

This patch introduces a coarse grained partition-level lock
ScopedPartitionLock to prevent dirty writes for multi-row transactions,
similar to the ScopedRowLock, but for locking the entire LockManager
instead of individual rows.

A partition lock can only be held by a single transaction at a time. A
given transaction can acquire the lock for multiple times. To prevent
deadlocks, a wait-die scheme is used -- if a transaction requires a lock
held by another transaction:
1. Retry the op if the requesting transaction has a lower txn ID than
   the current holder ("wait"),
2. Otherwise abort the requesting transaction immediately ("die").

A later patch will plumb this locking into participant ops and
transactional write ops.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 516 insertions(+), 16 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/11
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 11
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Bankim Bhavsar, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#6).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction (or write op) at a time, the same
transaction can acquire the lock for multiple times. To prevent deadlock,
'wait-die' scheme is used, which if the transaction requires a lock held by
another transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 457 insertions(+), 13 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/6
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 14: Code-Review+2

LGTM!

IWYU isn't happy yet, though


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 14
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 15:25:07 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 15: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 15
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 18:10:16 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(8 comments)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@158
PS6, Line 158:   ASSERT_EQ(1, lock_manager_.partition_lock_refs());
Looking at this as a block box, does it make sense to check that the first lock is still held even if the other went out of the scope?


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@252
PS6, Line 252: if (acquired) {
Why it might not be acquired?  Is it possible?


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager-test.cc@267
PS6, Line 267:     if (acquired) {
             :       CHECK_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :       CHECK_EQ(1, lock_manager_.partition_lock_refs());
             :     }
I'm not sure I understand this piece if reading the comment for the thread's functor: it seems the lock should not be ever acquired.  If so, why to check for acquired at all?


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.h@193
PS6, Line 193: ScopedPartitionLock
What if an instance of this class is copied?  Would it result in incorrect lock reference count for the underlying PartitionLock?


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@448
PS6, Line 448: kMaxBackoffExp
nit: this might be a constexpr


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@485
PS6, Line 485:     SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
> It's bad practice to sleep in a thread that is holding a lock. Is there any
BTW, why to hold that spinlock-based lock when sleeping (i.e., why not to release it before going to sleep) -- it's possible to re-acquire it on the next iteration of the loop.

Anywas, I guess using Semaphore::TimedAcquire() might be the best option as Andrew suggested.


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@496
PS6, Line 496: ms
Is it in seconds or milliseconds?

I guess the report might be skewed because thread might be put to sleep for longer.  Would it make sense just to use KLOG_EVERY_N_SECS() here instead, so the wait interval passed to TryAcquirePartitionLock() could be arbitrary?


http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@520
PS6, Line 520:   partition_lock_refs_ -= 1;
             :   DCHECK_GE(partition_lock_refs_, 0);
             :   if (partition_lock_refs_ == 0) {
             :     partition_lock_.reset();
             :   }
nit: this might be shortened to

DCHECK_GT(partition_lock_refs_, 0);
if (--partition_lock_refs == 0) {
  partition_lock_.reset();
}



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 03:51:58 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/6/src/kudu/tablet/lock_manager.cc@485
PS6, Line 485:     SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
> Hmm, we are only sleeping if we cannot hold the lock. But I will try to use
The point here is that the lock based on spinlock is taken at line 452 and hold when SleepFor() is executed at line 485.  This is a no-no.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 6
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Fri, 12 Mar 2021 04:30:41 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#2).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coase grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction at a time, the same transaction
can hold the lock for multiple times. To prevent deadlock, 'wait-die'
scheme is used, which if the transaction requires a lock held by another
transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 279 insertions(+), 7 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/2
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 2
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/17097

to look at the new patch set (#3).

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................

KUDU-2612: add PartitionLock and ScopedPartitionLock

This patch introduces a coarse grained partition level lock, PartitionLock
to prevent dirty writes for multi-rows transaction. A partition lock can
only be held by a single transaction at a time, the same transaction
can hold the lock for multiple times. To prevent deadlock, 'wait-die'
scheme is used, which if the transaction requires a lock held by another
transaction,
  1) abort the transaction immediately if it has a higher txn ID than
     the other one.
  2) Otherwise, retry the write op (or participant op) of the transaction.

A ScopedPartitionLock is also introduced to automatically manage the
lifecycle of a PartitionLock.

Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
---
M src/kudu/tablet/lock_manager-test.cc
M src/kudu/tablet/lock_manager.cc
M src/kudu/tablet/lock_manager.h
M src/kudu/tserver/tserver.proto
4 files changed, 278 insertions(+), 12 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/97/17097/3
-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 3
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 2:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@159
PS2, Line 159: TEST_F(LockManagerTest, TestLockUnlockPartitionMultiTxn) {
             :   // Acquiring a lock that held by another transaction which has a lower txn ID
             :   // will get a 'TXN_LOCKED_ABORT' server error code.
             :   {
             :     TxnId txn1(0);
             :     ScopedPartitionLock first_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn1);
             :     TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
             :     ASSERT_TRUE(first_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :     ASSERT_EQ(1, lock_manager_.partition_lock_refs());
             : 
             :     TxnId txn2(1);
             :     ScopedPartitionLock second_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn2);
             :     ASSERT_FALSE(second_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::TXN_LOCKED_ABORT, code);
             :   }
nit: probably makes sense to split TXN_LOCKED_ABORT code testing into a separate test case.


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager-test.cc@179
PS2, Line 179:     TxnId txn1(1);
             :     ScopedPartitionLock first_lock(&lock_manager_, LockManager::LOCK_EXCLUSIVE, txn1);
             :     TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
             :     ASSERT_TRUE(first_lock.Acquired(&code));
             :     ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
             :     ASSERT_EQ(1, lock_manager_.partition_lock_refs());
             : 
             :     TxnId txn2(0);
nit: the values of these transaction IDs are pretty confusing IMO. Might be easier to reason about if txn1 has value 1, txn2 has value 2, etc. Or at the very least, maintain ordering?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@368
PS2, Line 368: code_
Why bother storing this at all if we evaluate it every time we create a ScopedPartitionLock? Why not just call AcquirePartitionLock() when we call Acquired()?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@392
PS2, Line 392: 
             : ScopedPartitionLock::ScopedPartitionLock(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             : }
             : 
             : ScopedPartitionLock& ScopedPartitionLock::operator=(ScopedPartitionLock&& other) noexcept {
             :   TakeState(&other);
             :   return *this;
             : }
Curious, how do we expect to use these? It isn't clear, but that'd help gauge the correctness of TakeState().


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tablet/lock_manager.cc@422
PS2, Line 422: PartitionLock* LockManager::AcquirePartitionLock(
             :     const TxnId& txn_id,
             :     TabletServerErrorPB::Code* code) {
What happens if a non-transaction attempts to take the lock while a transaction is on-going? Can you add a test for when this is called with an invalid txn ID, and what happens if that races with a non-invalid txn ID?


http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto
File src/kudu/tserver/tserver.proto:

http://gerrit.cloudera.org:8080/#/c/17097/2/src/kudu/tserver/tserver.proto@116
PS2, Line 116: transaction
nit: to be clear, the whole transaction doesn't need to be retried, just the write op, right?



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 2
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Mon, 22 Feb 2021 06:30:24 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add PartitionLock and ScopedPartitionLock

Posted by "Hao Hao (Code Review)" <ge...@cloudera.org>.
Hao Hao has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add PartitionLock and ScopedPartitionLock
......................................................................


Patch Set 5:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/17097/3//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/17097/3//COMMIT_MSG@12
PS3, Line 12: sact
> nit: acquire?
Done


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc@195
PS3, Line 195:   CountDownLatch latch(kNumStartAtOnce);
> nit: since lock acquisition is quite fast operation, maybe it's worth synch
Done


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager-test.cc@208
PS3, Line 208:   ASSERT_EQ(0, lock_manager_.partition_lock_refs());
> nit: since lock acquisition is quite fast operation, maybe it's worth synch
Done


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h
File src/kudu/tablet/lock_manager.h:

http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@19
PS3, Line 19: #include <cstdint>
> Nit: Can use the C++ compatible <cstdint> instead.
Done


http://gerrit.cloudera.org:8080/#/c/17097/3/src/kudu/tablet/lock_manager.h@157
PS3, Line 157:  multi-rows t
> I find it weird that a class named *Lock doesn't have any lock/unlock/relea
I found it is more clear to have a specific entity to represent the lock. Even though it is just a wrapper around TxnId. But it can be further extended to contain more required field if needed in future.



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 5
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Thu, 11 Mar 2021 07:21:28 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2612: add ScopedPartitionLock

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/17097 )

Change subject: KUDU-2612: add ScopedPartitionLock
......................................................................


Patch Set 14:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc
File src/kudu/tablet/lock_manager-test.cc:

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc@267
PS13, Line 267:           } else {
> nit: does it make sense to add an ASSERT to make sure the lock is either ac
Done


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager-test.cc@270
PS13, Line 270:         }
> Very nice scenario -- thank you for adding this one.
Thanks!


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc
File src/kudu/tablet/lock_manager.cc:

http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@485
PS13, Line 485:       bool has_timeout = timeout.Initialized();
> nit: is it possible to move this under the 'if (timeout.Initialized())' cas
Done


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@521
PS13, Line 521: 
              : PartitionLockState* LockManager::WaitUntilAcquiredPartitionLo
> nit: combine these two lines into a single one?
Done


http://gerrit.cloudera.org:8080/#/c/17097/13/src/kudu/tablet/lock_manager.cc@522
PS13, Line 522: edPartition
> nit: maybe, change the signature of this method to make this last parameter
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/17097
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I158115739ce3e7cfb77bbcb854e834336c1256b1
Gerrit-Change-Number: 17097
Gerrit-PatchSet: 14
Gerrit-Owner: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Bankim Bhavsar <ba...@cloudera.com>
Gerrit-Reviewer: Hao Hao <ha...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Comment-Date: Tue, 23 Mar 2021 07:43:13 +0000
Gerrit-HasComments: Yes