You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Alexey Serbin (Code Review)" <ge...@cloudera.org> on 2018/12/19 21:39:25 UTC

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Alexey Serbin has uploaded this change for review. ( http://gerrit.cloudera.org:8080/12112


Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to
for read-only access to the guarded entities of the ThreadMgr.
In addition, do not hold the ThreadMgr's lock for the whole duration
of generating the /threadz page for the embedded webserver.

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 70 insertions(+), 66 deletions(-)



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 1
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

With this patch, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while removing thread information from its registry.
The process of adding an information about a thread into the registry
is not guarded by stricter consistency checks (i.e. it stays as it was
before this patch); see below for explanation.

NOTE:
  The stricter consistency around adding a thread into the ThreadMgr's
  registry caused issues with multiprocessing in Python tests,
  and I spent some time trying to work around that.  However, that was
  not fruitful.  I think the proper solution would be keeping the thread
  registry bound to some top-level object (like Kudu client or
  ServerBase object) and cleaning it up in accordance with the object's
  life cycle.

  In essence, the problem happens due to the combination of the
  following:
    * The way how workers are spawned by the multiprocessing.Pool,
      i.e. calling fork() at the point where Pool is instantiated.
    * The fact that pthread_t handles might be the same for threads
      in different processes.

  In detail, if multiprocessing.Pool is spawning worker processes after
  an instance of Kudu client has been created in the main test process,
  every worker gets a copy of the thread registry in its address space.
  The unexpected copy of the registry in a worker process is populated
  with the information on threads spawned due to the activity of the
  Kudu client in the main test process.  Later on, if a worker
  instantiates a Kudu client on its own, the newly spawned threads by
  the worker's Kudu client might have the same pthread_t handles
  as the threads whose information records are inadvertently inherited
  from the main test process.

  Also, it turned out it's impossible to handle worker's crash in a
  multiprocessing.Pool, and that's by design: for details see:
    https://stackoverflow.com/questions/24894682/

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 133 insertions(+), 100 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 10
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 9:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12112/9/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/9/src/kudu/util/thread.cc@424
PS9, Line 424:     map<string, size_t> thread_categories_info;
unordered_map here too?

Or is the goal to sort by category name? If so, add a comment.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 9
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 03 Jan 2019 00:20:50 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 1:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@165
PS1, Line 165:     std::lock_guard<decltype(lock_)> l(lock_);
Is this actually necessary? No other thread should have access to ThreadMgr while some thread is running its destructor.


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@263
PS1, Line 263:   std::lock_guard<decltype(lock_)> l(lock_);
Why do we need this acquisition?


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@346
PS1, Line 346: void ThreadMgr::PrintThreadDescriptorRow(const ThreadDescriptor& desc,
DCHECK that lock_ isn't held?


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@402
PS1, Line 402:               "<h4>" << threads_running_metric_ << " thread(s) running"
Doesn't this access need to be protected by lock_?

BTW you might consider adding a unit test that starts/stop threads all while polling the web UI. I wonder if you could reuse PeriodicWebUIChecker for this.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 1
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Wed, 19 Dec 2018 22:23:17 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 3: Code-Review+2

Looks OK but it's suspicious that both DEBUG and RELEASE Python tests timed out.


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 3
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 16:31:54 +0000
Gerrit-HasComments: No

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 8:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12112/8//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12112/8//COMMIT_MSG@49
PS8, Line 49:   As an additional piece of joy, it turned out that it's impossible to
            :   handle worker's crash in a multiprocessing.Pool, and that's by design.
            :   When a worker process crashes while holding the call_queue.rlock,
            :   no other process will ever be able to read the call_queue anymore,
            :   breaking the Pool as it cannot communicate with its workers.
> Nice find. Is this the same as https://stackoverflow.com/questions/24894682
Yep, that's exactly it.  It's good they describe the alternative available in Python 3 to handle the case.


http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc@226
PS8, Line 226:   typedef map<string, ThreadCategory> ThreadCategoryMap;
> Should this also be unordered_map? Out of scope for this change?
Yep, I think that's a good idea.  Done.


http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc@335
PS8, Line 335:     thread_categories_[category][pthread_id] =
             :         ThreadDescriptor(category, name, tid);
> May be worth a comment saying why we don't EmplaceOrDie here (the bit about
Done



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 8
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 03 Jan 2019 00:09:54 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

In addition, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while adding/removing thread information from
its registry.

The stricter consistecy around the ThreadMgr's registry caused issues
with multiprocessing in Python, so I updated the code in
test_scantoken.py to address that:
  a) For python2.7 and later, start worker processes of the
     multiprocessing pool _before_ a Kudu client object is instantiated.
     Also, every worker process will terminate after executing a single
     task, so the pool will spawn a new process with fresh address
     space for follow-up tasks, if any.
  b) For python2.6, switch to the serial execution of scan operations
     originated from serialized tokens.

NOTE:
  If spawning worker processes after the setUpClass() method, each
  worker in multiprocessing pool gets a copy of the ThreadMgr's thread
  registry in its address space with information on threads spawned by
  the client from the main test process.  Later on, when a newly
  spawned multiprocessing worker creates an instance of a Kudu client
  class on its own, the OS might reuse the formerly used thread
  identifiers in the new (i.e. worker) process.  That would trigger
  the CHECKs on the consistency of the thread registry.

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M python/kudu/tests/test_scantoken.py
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
3 files changed, 162 insertions(+), 120 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 6
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

In addition, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while adding/removing thread information from
its registry.

The stricter consistecy around the ThreadMgr's registry caused issues
with multiprocessing in Python, so I updated the code in
test_scantoken.py to address that.  In essence, all worker processes
in the multiprocessing pool are started before the Kudu client in
KuduTestBase is initialized.

NOTE:
  If spawning worker processes after the setUpClass() method, each
  worker in multiprocessing pool gets a copy of the ThreadMgr's thread
  registry in its address space with information on threads spawned by
  the client from the main test process.  Later on, when a newly
  spawned multiprocessing worker creates an instance of a Kudu client
  class on its own, the OS might reuse the formerly used thread
  identifiers in the new (i.e. worker) process.  That would trigger
  the CHECKs on the consistency of the thread registry.

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M python/kudu/tests/test_scantoken.py
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
3 files changed, 146 insertions(+), 122 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 7
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 5:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12112/5//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12112/5//COMMIT_MSG@24
PS5, Line 24: might be related
            : to the fact that process and thread management exercised by python
            : multiprocessing seems a bit weird
> Those as well might be some implications related to the way how the child m
Yep, indeed -- the issue was related to the way how workers are spawned in Python multiprocessing.  I have a repro scenario which works on other OS as well.

Since those workers are spawned by fork(), the thread registry of already opened Kudu client copied over into the spawned processes.  Later on, the identifiers of threads coincides with the identifiers used in the parent process -- the records for the latter threads are in the ThreadMgr registry which was copied over by fork() while creating every worker in the multiprocessing pool, so an attempt to insert a duplicate record into the registry triggers an assert.

In Python 3 that has been resolved by using context and additional modes (e.g., 'forkserver') that can be used to resolve the problem.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 5
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Tue, 25 Dec 2018 23:42:51 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to
for read-only access to the guarded entities of the ThreadMgr.
In addition, do not hold the ThreadMgr's lock for the whole duration
of generating the /threadz page for the embedded webserver.

In addition, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while adding/removing thread information from
its registry.

The latter caused issues with python multiprocessing, so I updated
the code in test_scantoken.py to switch to serial execution of
scan operations originated from serialized tokens.  I haven't gotten
to the bottom of it, but I think the problem of not calling thread
cleanup routines upon exit/cancellation of a thread might be related
to the fact that process and thread management exercised by python
multiprocessing seems a bit weird. F or example, see the bug originated
from the fact that multiprocessing exits the spawned processed by
os._exit(), not sys.exit(): https://bugs.python.org/issue18966

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M python/kudu/tests/test_scantoken.py
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
3 files changed, 113 insertions(+), 115 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 5
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to
for read-only access to the guarded entities of the ThreadMgr.
In addition, do not hold the ThreadMgr's lock for the whole duration
of generating the /threadz page for the embedded webserver.

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 75 insertions(+), 73 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 3
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 10: Code-Review+2


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 10
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 03 Jan 2019 01:32:13 +0000
Gerrit-HasComments: No

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 4: Code-Review+2


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 4
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Fri, 21 Dec 2018 18:31:59 +0000
Gerrit-HasComments: No

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 2:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc@304
PS2, Line 304: void ThreadMgr::AddThread(const pthread_t& pthread_id, const string& name,
You sure it's OK to remove the annotations? Commit 5693f6910 added them under the assumption that without the annotations, TSAN won't find certain data races.


http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc@338
PS2, Line 338:   DCHECK(!lock_.is_locked());
I thought you said this isn't safe; there could be a concurrent caller adding or removing a thread, no?



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 2
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 05:23:53 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 9:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12112/9/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/9/src/kudu/util/thread.cc@424
PS9, Line 424:     map<string, size_t> thread_categories_info;
> unordered_map here too?
Ah, right -- the intent was to keep the list of categories at the /threads page sorted.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 9
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 03 Jan 2019 01:18:08 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 1:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@346
PS1, Line 346: void ThreadMgr::PrintThreadDescriptorRow(const ThreadDescriptor& desc,
> I thought that would be a good idea and added DCHECK(!lock_.is_locked()) he
Fair point; turns out DCHECK(lock_.is_locked()) and DCHECK(!lock_.is_locked()) are much different.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 1
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 04:49:50 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

With this patch, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while removing thread information from its registry.
The process of adding an information about a thread into the registry
is not guarded by stricter consistency checks (i.e. it stays as it was
before this patch); see below for explanation.

NOTE:
  The stricter consistency around adding a thread into the ThreadMgr's
  registry caused issues with multiprocessing in Python tests,
  and I spent some time trying to work around that.  However, that was
  not fruitful.  I think the proper solution would be keeping the thread
  registry bound to some top-level object (like Kudu client or
  ServerBase object) and cleaning it up in accordance with the object's
  life cycle.

  In essence, the problem happens due to the combination of the
  following:
    * The way how workers are spawned by the multiprocessing.Pool,
      i.e. calling fork() at the point where Pool is instantiated.
    * The fact that pthread_t handles might be the same for threads
      in different processes.

  In detail, if multiprocessing.Pool is spawning worker processes after
  an instance of Kudu client has been created in the main test process,
  every worker gets a copy of the thread registry in its address space.
  The unexpected copy of the registry in a worker process is populated
  with the information on threads spawned due to the activity of the
  Kudu client in the main test process.  Later on, if a worker
  instantiates a Kudu client on its own, the newly spawned threads by
  the worker's Kudu client might have the same pthread_t handles
  as the threads whose information records are inadvertently inherited
  from the main test process.

  Also, it turned out it's impossible to handle worker's crash in a
  multiprocessing.Pool, and that's by design: for details see:
    https://stackoverflow.com/questions/24894682/

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Reviewed-on: http://gerrit.cloudera.org:8080/12112
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 133 insertions(+), 100 deletions(-)

Approvals:
  Adar Dembo: Looks good to me, approved
  Kudu Jenkins: Verified

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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 11
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 1:

(4 comments)

Thank you for the review.  I updated the webserver-stress-itest to make it cover this piece of functionality, but so far it passes.  However, my attempts to expose the races if removing the locks at some places are not yet successful.  I'll post an update for the test when I have repro scenarios for anticipated TSAN races.

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@165
PS1, Line 165:     std::lock_guard<decltype(lock_)> l(lock_);
> Is this actually necessary? No other thread should have access to ThreadMgr
I also was curious regarding this lock, and decided to keep it because I suspected there were some TSAN-related complications if removing it :)

OK, let's remove this and see -- if something surfaces from this place, I'll fix it in a follow-up changelist.


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@263
PS1, Line 263:   std::lock_guard<decltype(lock_)> l(lock_);
> Why do we need this acquisition?
Good catch.  It seems this is not needed: nothing that should be guarded accessed directly in this method and also this method is being executed under std::once's implicit guard.


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@346
PS1, Line 346: void ThreadMgr::PrintThreadDescriptorRow(const ThreadDescriptor& desc,
> DCHECK that lock_ isn't held?
I thought that would be a good idea and added DCHECK(!lock_.is_locked()) here, but when running tests with multiple concurrent thread it failed: it seems that due to the concurrency that check triggered.  I tried to find a way to check whether this particular thread holds the lock, but I didn't find that.

I also looked at AssertWaitAllowed/AssertIOAllowed, but I'm not sure it possible to use those as we would want.

Please let me know if I'm missing something here.


http://gerrit.cloudera.org:8080/#/c/12112/1/src/kudu/util/thread.cc@402
PS1, Line 402:               "<h4>" << threads_running_metric_ << " thread(s) running"
> Doesn't this access need to be protected by lock_?
Ah, good catch: I need to move the shared_lock<> a few lines above.

Thanks, I'll add the test.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 1
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 02:31:43 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to
for read-only access to the guarded entities of the ThreadMgr.
In addition, do not hold the ThreadMgr's lock for the whole duration
of generating the /threadz page for the embedded webserver.

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 79 insertions(+), 85 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 4
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 3:

> Looks OK but it's suspicious that both DEBUG and RELEASE Python
 > tests timed out.

Yep, something is fishy -- I'll try to clarify on this.


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 3
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 18:53:36 +0000
Gerrit-HasComments: No

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

With this patch, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while removing thread information from its registry.
The process of adding an information about a thread into the registry
is not guarded by stricter consistency checks (i.e. it stays as it was
before this patch); see below for explanation.

NOTE:
  The stricter consistency around adding a thread into the ThreadMgr's
  registry caused issues with multiprocessing in Python tests,
  and I spent some time trying to work around that.  However, that was
  not fruitful.  I think the proper solution would be keeping the thread
  registry bound to some top-level object (like Kudu client or
  ServerBase object) and cleaning it up in accordance with the object's
  life cycle.

  In essence, the problem happens due to the combination of the
  following:
    * The way how workers are spawned by the multiprocessing.Pool,
      i.e. calling fork() at the point where Pool is instantiated.
    * The fact that pthread_t handles might be the same for threads
      in different processes.

  In detail, if multiprocessing.Pool is spawning worker processes after
  an instance of Kudu client has been created in the main test process,
  every worker gets a copy of the thread registry in its address space.
  The unexpected copy of the registry in a worker process is populated
  with the information on threads spawned due to the activity of the
  Kudu client in the main test process.  Later on, if a worker
  instantiates a Kudu client on its own, the newly spawned threads by
  the worker's Kudu client might have the same pthread_t handles
  as the threads whose information records are inadvertently inherited
  from the main test process.

  Also, it turned out it's impossible to handle worker's crash in a
  multiprocessing.Pool, and that's by design: for details see:
    https://stackoverflow.com/questions/24894682/

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 121 insertions(+), 97 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 9
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to deal
with write and read access to the guarded entities of the ThreadMgr.
In addition, do not hold the lock for the whole duration of generating
the /threadz page for the embedded webserver.

With this patch, ThreadMgr now uses std::unordered_map as a container
for category-specific thread information and enforces stricter
consistency rules while removing thread information from its registry.
The process of adding an information about a thread into the registry
is not guarded by stricter consistency checks (i.e. it stays as it was
before this patch); see below for explanation.

NOTE:
  The stricter consistency around adding a thread into the ThreadMgr's
  registry caused issues with multiprocessing in Python tests,
  and I spent some time trying to work around that.  However, that was
  not fruitful.  I think the proper solution would be keeping the thread
  registry bound to some top-level object (like Kudu client or
  ServerBase object) and cleaning it up in accordance with the object's
  life cycle.

  In essence, the problem happens due to the combination of the
  following:
    * The way how workers are spawned by the multiprocessing.Pool,
      i.e. calling fork() at the point where Pool is instantiated.
    * The fact that pthread_t handles might be the same for threads
      in different processes.

  In detail, if multiprocessing.Pool is spawning worker processes after
  an instance of Kudu client has been created in the main test process,
  every worker gets a copy of the thread registry in its address space.
  The unexpected copy of the registry in a worker process is populated
  with the information on threads spawned due to the activity of the
  Kudu client in the main test process.  Later on, if a worker
  instantiates a Kudu client on its own, the newly spawned treads by
  the worker's Kudu client might have the same pthread_t handles
  as the threads whose information records are inadvertently inherited
  from the main test process.

  As an additional piece of joy, it turned out that it's impossible to
  handle worker's crash in a multiprocessing.Pool, and that's by design.
  When a worker process crashes while holding the call_queue.rlock,
  no other process will ever be able to read the call_queue anymore,
  breaking the Pool as it cannot communicate with its workers.

  BTW, in Python 3 the problem with the duplicated address space
  in worker processes has been resolved by using context and additional
  worker spawn modes (e.g., 'forkserver').

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 101 insertions(+), 95 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 8
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 2:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc@304
PS2, Line 304: void ThreadMgr::AddThread(const pthread_t& pthread_id, const string& name,
> You sure it's OK to remove the annotations? Commit 5693f6910 added them und
Nope, I'm not sure.  Probably, I misunderstood the comments in the code.  After reading comments in the commit message for 5693f6910, I think we better keep them as is.

Sorry for misunderstanding.


http://gerrit.cloudera.org:8080/#/c/12112/2/src/kudu/util/thread.cc@338
PS2, Line 338:   DCHECK(!lock_.is_locked());
> I thought you said this isn't safe; there could be a concurrent caller addi
Ah, indeed.  It's some remnant from the earlier versions -- I'm working with two repos (local and remote), starting changes at the remote one sometimes.  Good catch -- I'll fix this, thanks.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 2
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Thu, 20 Dec 2018 05:32:20 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Hello Will Berkeley, Kudu Jenkins, Adar Dembo, 

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

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

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................

[util] use lighter locking scheme for ThreadMgr

Use the rw_spinlock primitive to guard the ThreadMgr's registry
threads instead of Mutex.  Also, use the shared lock pattern to
for read-only access to the guarded entities of the ThreadMgr.
In addition, do not hold the ThreadMgr's lock for the whole duration
of generating the /threadz page for the embedded webserver.

Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
---
M src/kudu/util/os-util.cc
M src/kudu/util/thread.cc
2 files changed, 82 insertions(+), 106 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 2
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 5:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/12112/5//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12112/5//COMMIT_MSG@24
PS5, Line 24: might be related
            : to the fact that process and thread management exercised by python
            : multiprocessing seems a bit weird
Those as well might be some implications related to the way how the child multiprocessing workers are spawned -- need to clarify on this.


http://gerrit.cloudera.org:8080/#/c/12112/5//COMMIT_MSG@26
PS5, Line 26:  
nit: remove



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 5
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Sat, 22 Dec 2018 23:47:27 +0000
Gerrit-HasComments: Yes

[kudu-CR] [util] use lighter locking scheme for ThreadMgr

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

Change subject: [util] use lighter locking scheme for ThreadMgr
......................................................................


Patch Set 8:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12112/8//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12112/8//COMMIT_MSG@49
PS8, Line 49:   As an additional piece of joy, it turned out that it's impossible to
            :   handle worker's crash in a multiprocessing.Pool, and that's by design.
            :   When a worker process crashes while holding the call_queue.rlock,
            :   no other process will ever be able to read the call_queue anymore,
            :   breaking the Pool as it cannot communicate with its workers.
Nice find. Is this the same as https://stackoverflow.com/questions/24894682/python-multiprocessing-crash-in-subprocess? If so, you could elide some of the explanation and link to that.


http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc
File src/kudu/util/thread.cc:

http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc@226
PS8, Line 226:   typedef map<string, ThreadCategory> ThreadCategoryMap;
Should this also be unordered_map? Out of scope for this change?


http://gerrit.cloudera.org:8080/#/c/12112/8/src/kudu/util/thread.cc@335
PS8, Line 335:     thread_categories_[category][pthread_id] =
             :         ThreadDescriptor(category, name, tid);
May be worth a comment saying why we don't EmplaceOrDie here (the bit about forking after creating a kudu::Thread and how pthread identifiers may be reused).



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I4d49c1c39392e01c45019844430a4fe3d116c277
Gerrit-Change-Number: 12112
Gerrit-PatchSet: 8
Gerrit-Owner: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Will Berkeley <wd...@gmail.com>
Gerrit-Comment-Date: Wed, 02 Jan 2019 17:39:16 +0000
Gerrit-HasComments: Yes