[
https://issues.apache.org/jira/browse/CASSANDRA-17085?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17437450#comment-17437450
]
David Capwell edited comment on CASSANDRA-17085 at 11/2/21, 4:18 PM:
---------------------------------------------------------------------
Here is what I see different than 4.0
{code}
$ diff 40.txt trunk.txt
4,5c4
< COMMIT-LOG-ALLOCATOR false
< CacheCleanupExecutor:1 true
---
> COMMIT-LOG-ALLOCATOR true
10d8
< GossipStage:1 true
12,14d9
< HintsDispatcher:1 true
< HintsDispatcher:2 true
< HintsWriteExecutor:1 true
23d17
< MigrationStage:1 true
27c21
< PERIODIC-COMMIT-LOG-SYNCER false
---
> PERIODIC-COMMIT-LOG-SYNCER true
37d30
< SecondaryIndexManagement:1 true
40,42c33
< ValidationExecutor:1 true
< ValidationExecutor:2 true
< ViewBuildExecutor:1 true
---
> SnapshotCleanup:1 true
45a37
> read-hotness-tracker:1 true
{code}
This was collected as the last step in main, so just scans every thread and
checks if daemon or not
the non-daemon threads in 40 are: nioEventLoopGroup-5-1,
PERIODIC-COMMIT-LOG-SYNCER, and COMMIT-LOG-ALLOCATOR
but in trunk it is only: nioEventLoopGroup-5-1
seems PERIODIC-COMMIT-LOG-SYNCER and COMMIT-LOG-ALLOCATOR went daemon; will try
to track down where.
PeriodicCommitLogService:
40: NamedThreadFactory.createThread(new
SyncRunnable(MonotonicClock.preciseTime), name) // daemon=false
trunk: executorFactory().infiniteLoop(name, new
SyncRunnable(MonotonicClock.preciseTime), true); // daemon=true in
org.apache.cassandra.concurrent.ExecutorFactory.Default#startThread
AbstractCommitLogSegmentManager
40: NamedThreadFactory.createThread(runnable, "COMMIT-LOG-ALLOCATOR"); //
daemon=false
trunk: executorFactory().infiniteLoop("COMMIT-LOG-ALLOCATOR", runnable, true,
interruptHandler); // daemon=true
These two were changed in CASSANDRA-16925
was (Author: dcapwell):
Here is what I see different than 4.0
{code}
$ diff 40.txt trunk.txt
4,5c4
< COMMIT-LOG-ALLOCATOR false
< CacheCleanupExecutor:1 true
---
> COMMIT-LOG-ALLOCATOR true
10d8
< GossipStage:1 true
12,14d9
< HintsDispatcher:1 true
< HintsDispatcher:2 true
< HintsWriteExecutor:1 true
23d17
< MigrationStage:1 true
27c21
< PERIODIC-COMMIT-LOG-SYNCER false
---
> PERIODIC-COMMIT-LOG-SYNCER true
37d30
< SecondaryIndexManagement:1 true
40,42c33
< ValidationExecutor:1 true
< ValidationExecutor:2 true
< ViewBuildExecutor:1 true
---
> SnapshotCleanup:1 true
45a37
> read-hotness-tracker:1 true
{code}
This was collected as the last step in main, so just scans every thread and
checks if daemon or not
the non-daemon threads in 40 are: nioEventLoopGroup-5-1,
PERIODIC-COMMIT-LOG-SYNCER, and COMMIT-LOG-ALLOCATOR
but in trunk it is only: nioEventLoopGroup-5-1
seems PERIODIC-COMMIT-LOG-SYNCER and COMMIT-LOG-ALLOCATOR went daemon; will try
to track down where.
> Fix python dtests bootstrap_test.py::TestBootstrap
> --------------------------------------------------
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
> Issue Type: Bug
> Components: Test/dtest/python
> Reporter: David Capwell
> Assignee: David Capwell
> Priority: Normal
> Fix For: 4.x
>
>
> Right now bootstrap tests are failing every time we run, this work is to
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> > node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014:
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
> _
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost',
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
> _
> process = <subprocess.Popen object at 0x7fb071a03940>
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
>
> if rc != 0:
> > raise ToolError(cmd_args, rc, out, err)
> E ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost',
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit
> status: 1;
> E stderr: nodetool: Failed to connect to 'localhost:7300' -
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> > node1.start()
> bootstrap_test.py:483:
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
> _
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout,
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
> _
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> > raise TimeoutError.create(start, timeout, msg, node)
> E ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in
> system.log:
> E Head:
> E Tail: ...
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]