Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Empty file added .attach_pid1466907
Empty file.
4 changes: 4 additions & 0 deletions .build/cassandra-deps-maven-pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,10 @@
<groupId>com.github.jbellis</groupId>
<artifactId>jamm</artifactId>
</dependency>
<dependency>
<groupId>tools.profiler</groupId>
<artifactId>async-profiler</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
Expand Down
6 changes: 6 additions & 0 deletions .build/parent-maven-pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
<byteman.version>4.0.20</byteman.version>
<netty.version>4.1.125.Final</netty.version>
<ohc.version>0.5.1</ohc.version>
<async-profiler.version>4.2</async-profiler.version>

<!-- These are referenced in build.xml, so need to be propagated from there -->
<asm.version>@asm.version@</asm.version>
Expand Down Expand Up @@ -465,6 +466,11 @@
<artifactId>jamm</artifactId>
<version>${jamm.version}</version>
</dependency>
<dependency>
<groupId>tools.profiler</groupId>
<artifactId>async-profiler</artifactId>
<version>${async-profiler.version}</version>
</dependency>
<dependency>
<!-- Test scoped jackson-dataformat-yaml also depends on snakeyaml. For now, these versions are aligned
but if you happen to bump it here then exclude it in jackson-dataformat-yaml.
Expand Down
1 change: 1 addition & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
5.1
* Support low-overhead async profiling (CASSANDRA-20854)
* Reject PrepareJoin if tokens are already assigned (CASSANDRA-21006)
* Don't update registration status if node state for decommissioned peer is found with the same address (CASSANDRA-21005)
* Avoid NPE when meta keyspace placements are empty before CMS is initialized (CASSANDRA-21004)
Expand Down
1 change: 1 addition & 0 deletions NEWS.txt
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,7 @@ New features
CASSANDRA-20102 adds a subset of the SQL99 (binary) string functions: "octet_length" defined on all types
and "length" defined on UTF8 strings. See CASSANDRA-20102 for more information.
- New functions `format_bytes` and `format_time` were added. See CASSANDRA-19546.
- It is possible to use Async-profiler for various profiling scenarios. See CASSANDRA-20854.

Upgrading
---------
Expand Down
7 changes: 7 additions & 0 deletions conf/jvm-server.options
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,13 @@
# Disable chronicle analytics. See CASSANDRA-19656
-Dchronicle.analytics.disable=true

# AsyncProfiler Flags
#-Dcassandra.async_profiler.enabled=true|false
#-Dcassandra.async_profiler.unsafe_mode=true|false
# When not set, logs from profiler will be placed under
# directory behind "cassandra.logdir" property, into "async-profiler" sub-directory
#-Dcassandra.logdir.async_profiler=/tmp/cassandra-profiling

### Debug options

# uncomment to enable flight recorder
Expand Down
1 change: 1 addition & 0 deletions doc/modules/cassandra/nav.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
**** xref:cassandra:managing/operating/password_validation.adoc[Password validation]
**** xref:cassandra:managing/operating/role_name_generation.adoc[Role name generation]
**** xref:cassandra:managing/operating/onboarding-to-accord.adoc[]
**** xref:cassandra:managing/operating/async-profiler.adoc[]
*** xref:cassandra:managing/tools/index.adoc[Tools]
**** xref:cassandra:managing/tools/cqlsh.adoc[cqlsh: the CQL shell]
**** xref:cassandra:managing/tools/nodetool/nodetool.adoc[nodetool]
Expand Down
103 changes: 103 additions & 0 deletions doc/modules/cassandra/pages/managing/operating/async-profiler.adoc
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
= Async-profiler

Since https://issues.apache.org/jira/browse/CASSANDRA-20854[CASSANDRA-20854], it is possible to use
https://github.com/async-profiler/async-profiler[async-profiler] to profile your nodes. Async-profiler is
shipped with Cassandra, so you do not need to do anything else but start to use it by enabling a property.
Async-profiler functionality is disabled by default. It can be turned on by setting Cassandra's `cassandra.async_profiler.enabled` property to `true`.

There is a command in `nodetool` called `profile` with these sub-commands:

== start

Basic usage:
----
$ nodetool profile start
----

This will start profiling, by default for 60 seconds. If you want, for example, profile memory allocations for
5 minutes and save results into a file memory-allocation-5m.html you would do:

----
$ nodetool profile start -e alloc -d 5m -o memory-allocation-5m.html
----

There are these events possible to profile:

'cpu', 'alloc', 'lock', 'wall', 'nativemem', 'cache_misses', delimited by comma, defaults to 'cpu'.

There are these output formats possible to specify, via `--format` flag:

'flat', 'traces', 'collapsed', 'flamegraph', 'tree', 'jfr', 'otlp', defaults to 'flamegraph'

== status

You can then inspect the state of profiling by `status` subcommand:

----
$ nodetool profile status
Profiling is running for 7 seconds
----

If you attempt to start another profiling while the current profiling is running, this will not be possible:

----
$ nodetool profile start -e alloc -d 5m -o memory-allocation-5m.html
Profiler has already started or there was a failure to start it.
----

== stop

You can stop the profiling prematurely by `stop` sub-command

----
$ nodetool profile stop -o memory-allocation-5m.html
----

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:

== list

----
$ nodetool profile list
memory-allocation-5m.html
cpu.html
----

== fetch

If you have access to a node, you can just go to, by default,
`logs` directory of Cassandra, into `async-profiler` and obtain a respective file. However, in a scenario when
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:

----
$ nodetool profile fetch cpu.html /tmp/cpu.html
----

== purge

Of course, more you profile, more disk space the results will occupy. If you have direct access, you can just
remove the files yourself, however if you do not, you need to use `purge` sub-command which will remove all profiling files:

----
$ nodetool profile purge
$ nodetool profile list
<no output>
----

== execute

You can also execute arbitrary commands, by `execute` subcommand, like this:

----
nodetool profile execute meminfo
Call trace storage: 10244 KB
Flight recording: 0 KB
Dictionaries: 68 KB
Code cache: 11934 KB
------------------------------
Total: 22246 KB
----

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.

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.
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,9 @@ public enum CassandraRelevantProperties
ALLOW_UNSAFE_REPLACE("cassandra.allow_unsafe_replace"),
ALLOW_UNSAFE_TRANSIENT_CHANGES("cassandra.allow_unsafe_transient_changes"),
APPROXIMATE_TIME_PRECISION_MS("cassandra.approximate_time_precision_ms", "2"),
ASYNC_PROFILER_ENABLED("cassandra.async_profiler.enabled", "false"),
ASYNC_PROFILER_LOG_DIR("cassandra.logdir.async_profiler"),
ASYNC_PROFILER_UNSAFE_MODE("cassandra.async_profiler.unsafe_mode", "false"),
/** 2 ** GENSALT_LOG2_ROUNDS rounds of hashing will be performed. */
AUTH_BCRYPT_GENSALT_LOG2_ROUNDS("cassandra.auth_bcrypt_gensalt_log2_rounds", "4"),
/** We expect default values on cache retries and interval to be sufficient for everyone but have this escape hatch just in case. */
Expand Down
127 changes: 127 additions & 0 deletions src/java/org/apache/cassandra/profiler/AsyncProfiler.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.cassandra.profiler;

import java.util.List;
import javax.management.StandardMBean;

import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.service.AsyncProfilerService;
import org.apache.cassandra.utils.MBeanWrapper;

import static org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_UNSAFE_MODE;

public abstract class AsyncProfiler implements AsyncProfilerMBean
{
protected final AsyncProfilerService service = new AsyncProfilerService();

private static AsyncProfiler instance;

public static synchronized AsyncProfiler instance()
{
if (AsyncProfiler.instance == null)
{
try
{
AsyncProfiler.instance = ASYNC_PROFILER_UNSAFE_MODE.getBoolean() ? new AsyncProfilerUnsafe() : new AsyncProfilerSafe();

// register mbean first, before initialisation, which might fail (e.g. profiler functionality is disabled)
MBeanWrapper.instance.registerMBean(new StandardMBean(AsyncProfiler.instance, AsyncProfilerMBean.class),
AsyncProfiler.MBEAN_NAME,
MBeanWrapper.OnException.LOG);

instance.initialize();
}
catch (ConfigurationException ex)
{
throw ex;
}
catch (IllegalStateException ex)
{
if (!"Async-Profiler is not enabled.".equals(ex.getMessage()))
throw ex;
}
catch (Throwable t)
{
throw new RuntimeException(t);
}
}

return AsyncProfiler.instance;
}

@Override
public boolean start(String events, String outputFormat, String duration, String outputFileName)
{
return service.start(events, outputFormat, duration, outputFileName);
}

@Override
public boolean stop(String outputFileName)
{
return service.stop(outputFileName);
}

@Override
public boolean isEnabled()
{
return service.isEnabled();
}

@Override
public void disable()
{
service.disable();
}

@Override
public void enable()
{
service.enable();
}

@Override
public void purge()
{
service.purge();
}

@Override
public List<String> list()
{
return service.list();
}

@Override
public byte[] fetch(String resultFile)
{
return service.fetch(resultFile);
}

@Override
public String status()
{
return service.status();
}

public void initialize()
{
service.maybeInitialize();
}
}
Loading