Skip to content

Commit 52c1ed0

Browse files
yziadehbbotella
authored andcommitted
Initial async-profiler Nodetool implementation
Add enable/disable AsyncProfiler JVM flags Refactor Add output format option & initial tests Refactor & add tests Add tests for system properties & refactor Checkstyle fixes & switching airlift with picocli (CASSANDRA-17445) Add more tests Fix Picocli 'Profile' command integration Address feedback Changes.txt Remove not needed property Add missing licenses Apply feedback Fix help tests refactoring fix commands hardened validation and simplification of commands fix log dir fixed tests more fixes add purge command more hardening implement list and fetch Remove unused import more fixes added documentation fixed nodetool help output tests added status command introduced binary download of files in fetch command if necessary hardened code ability to specify duration in human format (e.g. 5m) improved error parsing ability to execute purge, list and fetch even with disabled profiler async-profiler is disabled by default added nodetool tests add startup check checking kernel parameters
1 parent 209288c commit 52c1ed0

30 files changed

+1876
-0
lines changed

.build/cassandra-deps-maven-pom.xml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -161,6 +161,10 @@
161161
<groupId>com.github.jbellis</groupId>
162162
<artifactId>jamm</artifactId>
163163
</dependency>
164+
<dependency>
165+
<groupId>tools.profiler</groupId>
166+
<artifactId>async-profiler</artifactId>
167+
</dependency>
164168
<dependency>
165169
<groupId>io.netty</groupId>
166170
<artifactId>netty-all</artifactId>

.build/parent-maven-pom.xml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@
4040
<byteman.version>4.0.20</byteman.version>
4141
<netty.version>4.1.125.Final</netty.version>
4242
<ohc.version>0.5.1</ohc.version>
43+
<async-profiler.version>4.2</async-profiler.version>
4344

4445
<!-- These are referenced in build.xml, so need to be propagated from there -->
4546
<asm.version>@asm.version@</asm.version>
@@ -465,6 +466,11 @@
465466
<artifactId>jamm</artifactId>
466467
<version>${jamm.version}</version>
467468
</dependency>
469+
<dependency>
470+
<groupId>tools.profiler</groupId>
471+
<artifactId>async-profiler</artifactId>
472+
<version>${async-profiler.version}</version>
473+
</dependency>
468474
<dependency>
469475
<!-- Test scoped jackson-dataformat-yaml also depends on snakeyaml. For now, these versions are aligned
470476
but if you happen to bump it here then exclude it in jackson-dataformat-yaml.

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
5.1
2+
* Support low-overhead async profiling (CASSANDRA-20854)
23
* Reject PrepareJoin if tokens are already assigned (CASSANDRA-21006)
34
* Don't update registration status if node state for decommissioned peer is found with the same address (CASSANDRA-21005)
45
* Avoid NPE when meta keyspace placements are empty before CMS is initialized (CASSANDRA-21004)

NEWS.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -135,6 +135,7 @@ New features
135135
CASSANDRA-20102 adds a subset of the SQL99 (binary) string functions: "octet_length" defined on all types
136136
and "length" defined on UTF8 strings. See CASSANDRA-20102 for more information.
137137
- New functions `format_bytes` and `format_time` were added. See CASSANDRA-19546.
138+
- It is possible to use Async-profiler for various profiling scenarios. See CASSANDRA-20854.
138139

139140
Upgrading
140141
---------

conf/jvm-server.options

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -156,6 +156,13 @@
156156
# Disable chronicle analytics. See CASSANDRA-19656
157157
-Dchronicle.analytics.disable=true
158158

159+
# AsyncProfiler Flags
160+
#-Dcassandra.async_profiler.enabled=true|false
161+
#-Dcassandra.async_profiler.unsafe_mode=true|false
162+
# When not set, logs from profiler will be placed under
163+
# directory behind "cassandra.logdir" property, into "async-profiler" sub-directory
164+
#-Dcassandra.logdir.async_profiler=/tmp/cassandra-profiling
165+
159166
### Debug options
160167

161168
# uncomment to enable flight recorder

doc/modules/cassandra/nav.adoc

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -112,6 +112,7 @@
112112
**** xref:cassandra:managing/operating/password_validation.adoc[Password validation]
113113
**** xref:cassandra:managing/operating/role_name_generation.adoc[Role name generation]
114114
**** xref:cassandra:managing/operating/onboarding-to-accord.adoc[]
115+
**** xref:cassandra:managing/operating/async-profiler.adoc[]
115116
*** xref:cassandra:managing/tools/index.adoc[Tools]
116117
**** xref:cassandra:managing/tools/cqlsh.adoc[cqlsh: the CQL shell]
117118
**** xref:cassandra:managing/tools/nodetool/nodetool.adoc[nodetool]
Lines changed: 103 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,103 @@
1+
= Async-profiler
2+
3+
Since https://issues.apache.org/jira/browse/CASSANDRA-20854[CASSANDRA-20854], it is possible to use
4+
https://github.com/async-profiler/async-profiler[async-profiler] to profile your nodes. Async-profiler is
5+
shipped with Cassandra, so you do not need to do anything else but start to use it by enabling a property.
6+
Async-profiler functionality is disabled by default. It can be turned on by setting Cassandra's `cassandra.async_profiler.enabled` property to `true`.
7+
8+
There is a command in `nodetool` called `profile` with these sub-commands:
9+
10+
== start
11+
12+
Basic usage:
13+
----
14+
$ nodetool profile start
15+
----
16+
17+
This will start profiling, by default for 60 seconds. If you want, for example, profile memory allocations for
18+
5 minutes and save results into a file memory-allocation-5m.html you would do:
19+
20+
----
21+
$ nodetool profile start -e alloc -d 5m -o memory-allocation-5m.html
22+
----
23+
24+
There are these events possible to profile:
25+
26+
'cpu', 'alloc', 'lock', 'wall', 'nativemem', 'cache_misses', delimited by comma, defaults to 'cpu'.
27+
28+
There are these output formats possible to specify, via `--format` flag:
29+
30+
'flat', 'traces', 'collapsed', 'flamegraph', 'tree', 'jfr', 'otlp', defaults to 'flamegraph'
31+
32+
== status
33+
34+
You can then inspect the state of profiling by `status` subcommand:
35+
36+
----
37+
$ nodetool profile status
38+
Profiling is running for 7 seconds
39+
----
40+
41+
If you attempt to start another profiling while the current profiling is running, this will not be possible:
42+
43+
----
44+
$ nodetool profile start -e alloc -d 5m -o memory-allocation-5m.html
45+
Profiler has already started or there was a failure to start it.
46+
----
47+
48+
== stop
49+
50+
You can stop the profiling prematurely by `stop` sub-command
51+
52+
----
53+
$ nodetool profile stop -o memory-allocation-5m.html
54+
----
55+
56+
After the profiling is finished, either by waiting until it stops on its own or by us explicitly, we have a result file in a results directory on a node. We can inspect what results there are by `list` sub-command:
57+
58+
== list
59+
60+
----
61+
$ nodetool profile list
62+
memory-allocation-5m.html
63+
cpu.html
64+
----
65+
66+
== fetch
67+
68+
If you have access to a node, you can just go to, by default,
69+
`logs` directory of Cassandra, into `async-profiler` and obtain a respective file. However, in a scenario when
70+
you are executing remote profiling (nodetool exection is on a physically different machine from Cassandra node), or you do not have the direct access to remote disk, you need to use `fetch` subcommand, which will sent the content of your result file locally where you can save it to whatever destination you want:
71+
72+
----
73+
$ nodetool profile fetch cpu.html /tmp/cpu.html
74+
----
75+
76+
== purge
77+
78+
Of course, more you profile, more disk space the results will occupy. If you have direct access, you can just
79+
remove the files yourself, however if you do not, you need to use `purge` sub-command which will remove all profiling files:
80+
81+
----
82+
$ nodetool profile purge
83+
$ nodetool profile list
84+
<no output>
85+
----
86+
87+
== execute
88+
89+
You can also execute arbitrary commands, by `execute` subcommand, like this:
90+
91+
----
92+
nodetool profile execute meminfo
93+
Call trace storage: 10244 KB
94+
Flight recording: 0 KB
95+
Dictionaries: 68 KB
96+
Code cache: 11934 KB
97+
------------------------------
98+
Total: 22246 KB
99+
----
100+
101+
However, to execute arbitrary commands for Async-profiler, we need to enable _unsafe_ async profiling by system property of Cassandra `cassandra.async_profiler.unsafe_mode` set to `true`. You will not be able to do this otherwise.
102+
103+
You can also control where profiling files go via `cassandra.logdir.async_profiler` system property. When not set, by default they will be stored to `cassandra.logdir` + `async-profiler` directory.

src/java/org/apache/cassandra/config/CassandraRelevantProperties.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,9 @@ public enum CassandraRelevantProperties
6060
ALLOW_UNSAFE_REPLACE("cassandra.allow_unsafe_replace"),
6161
ALLOW_UNSAFE_TRANSIENT_CHANGES("cassandra.allow_unsafe_transient_changes"),
6262
APPROXIMATE_TIME_PRECISION_MS("cassandra.approximate_time_precision_ms", "2"),
63+
ASYNC_PROFILER_ENABLED("cassandra.async_profiler.enabled", "false"),
64+
ASYNC_PROFILER_LOG_DIR("cassandra.logdir.async_profiler"),
65+
ASYNC_PROFILER_UNSAFE_MODE("cassandra.async_profiler.unsafe_mode", "false"),
6366
/** 2 ** GENSALT_LOG2_ROUNDS rounds of hashing will be performed. */
6467
AUTH_BCRYPT_GENSALT_LOG2_ROUNDS("cassandra.auth_bcrypt_gensalt_log2_rounds", "4"),
6568
/** We expect default values on cache retries and interval to be sufficient for everyone but have this escape hatch just in case. */
Lines changed: 127 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,127 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.cassandra.profiler;
20+
21+
import java.util.List;
22+
import javax.management.StandardMBean;
23+
24+
import org.apache.cassandra.exceptions.ConfigurationException;
25+
import org.apache.cassandra.service.AsyncProfilerService;
26+
import org.apache.cassandra.utils.MBeanWrapper;
27+
28+
import static org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_UNSAFE_MODE;
29+
30+
public abstract class AsyncProfiler implements AsyncProfilerMBean
31+
{
32+
protected final AsyncProfilerService service = new AsyncProfilerService();
33+
34+
private static AsyncProfiler instance;
35+
36+
public static synchronized AsyncProfiler instance()
37+
{
38+
if (AsyncProfiler.instance == null)
39+
{
40+
try
41+
{
42+
AsyncProfiler.instance = ASYNC_PROFILER_UNSAFE_MODE.getBoolean() ? new AsyncProfilerUnsafe() : new AsyncProfilerSafe();
43+
44+
// register mbean first, before initialisation, which might fail (e.g. profiler functionality is disabled)
45+
MBeanWrapper.instance.registerMBean(new StandardMBean(AsyncProfiler.instance, AsyncProfilerMBean.class),
46+
AsyncProfiler.MBEAN_NAME,
47+
MBeanWrapper.OnException.LOG);
48+
49+
instance.initialize();
50+
}
51+
catch (ConfigurationException ex)
52+
{
53+
throw ex;
54+
}
55+
catch (IllegalStateException ex)
56+
{
57+
if (!"Async-Profiler is not enabled.".equals(ex.getMessage()))
58+
throw ex;
59+
}
60+
catch (Throwable t)
61+
{
62+
throw new RuntimeException(t);
63+
}
64+
}
65+
66+
return AsyncProfiler.instance;
67+
}
68+
69+
@Override
70+
public boolean start(String events, String outputFormat, String duration, String outputFileName)
71+
{
72+
return service.start(events, outputFormat, duration, outputFileName);
73+
}
74+
75+
@Override
76+
public boolean stop(String outputFileName)
77+
{
78+
return service.stop(outputFileName);
79+
}
80+
81+
@Override
82+
public boolean isEnabled()
83+
{
84+
return service.isEnabled();
85+
}
86+
87+
@Override
88+
public void disable()
89+
{
90+
service.disable();
91+
}
92+
93+
@Override
94+
public void enable()
95+
{
96+
service.enable();
97+
}
98+
99+
@Override
100+
public void purge()
101+
{
102+
service.purge();
103+
}
104+
105+
@Override
106+
public List<String> list()
107+
{
108+
return service.list();
109+
}
110+
111+
@Override
112+
public byte[] fetch(String resultFile)
113+
{
114+
return service.fetch(resultFile);
115+
}
116+
117+
@Override
118+
public String status()
119+
{
120+
return service.status();
121+
}
122+
123+
public void initialize()
124+
{
125+
service.maybeInitialize();
126+
}
127+
}

0 commit comments

Comments
 (0)