pauloricardomg commented on code in PR #4255:
URL: https://github.com/apache/cassandra/pull/4255#discussion_r2302015065


##########
src/java/org/apache/cassandra/profiler/AsyncProfilerMBean.java:
##########
@@ -0,0 +1,8 @@
+package org.apache.cassandra.profiler;
+
+public interface AsyncProfilerMBean {
+    void start(String event, String outputFormat);
+    void stop(String outputFile);

Review Comment:
   `outputFile` should not be part of the interface, since the nodetool caller 
does not know where the server is deployed and it can represent a security 
issue since it will allow a remote caller to create a file in the server host 
in a specific location. I think the output directory for async profiler should 
be specified in the  server configuration and the server should generate a file 
with name based on the execution timestamp.



##########
src/java/org/apache/cassandra/tools/nodetool/Profile.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.cassandra.tools.nodetool;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.profiler.AsyncProfilerMBean;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+@Command(name = "profile", description = "Run Async-Profiler on the cassandra 
process")
+public class Profile extends AbstractCommand {
+
+    @Option(names = {"-s", "--start"}, description = "Start profiling")
+    public boolean start;
+
+    @Option(names = {"-x", "--stop"}, description = "Stop profiling and dump 
output")
+    public boolean stop;
+
+    @Option(names = {"-e", "--event"}, description = "Event to profile (cpu, 
alloc, lock, wall, etc.)")
+    public String event = "cpu";
+
+    @Option(names = {"-r", "--raw"}, description = "Raw commands to execute")
+    public String raw;
+
+    @Option(names = {"-o", "--output"}, description = "Output file for profile 
dump")
+    public String outputFile = "/tmp/profile.html";
+
+    @Option(names = {"-f", "--format"}, description = "Output format 
(flamegraph, tree, traces, etc.)")
+    public String outputFormat = "flamegraph";
+
+    @Override
+    public void execute(NodeProbe probe) {
+        AsyncProfilerMBean profiler = probe.getAsyncProfilerProxy();
+        if (!profiler.isAvailable()) {
+            System.err.println("Async-profiler native library is not loaded or 
unavailable.");
+            return;
+        }
+
+        try {
+            if (start) {

Review Comment:
   all Cassandra braces should be on a newline, throughout all files



##########
src/java/org/apache/cassandra/tools/profiler/AsyncProfilerService.java:
##########
@@ -0,0 +1,111 @@
+package org.apache.cassandra.tools.profiler;
+
+import one.profiler.AsyncProfiler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File; //checkstyle: permit this import
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_LIB_PATH;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_ENABLED;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_ADVANCED_MODE;
+
+public class AsyncProfilerService {
+    private static final Logger logger = 
LoggerFactory.getLogger(AsyncProfilerService.class);
+
+    private static final Set<String> VALID_EVENTS = Set.of("cpu", "alloc", 
"lock", "wall", "nativemem", "cache-misses");
+    private static final Set<String> VALID_FORMATS = 
Set.of("flat","traces","collapsed","flamegraph","tree","jfr","otlp");
+    private static final Character[] INVALID_OUTPUT_FILENAME_CHARS = {'"', 
'*', '<', '>', '?', '|'};
+
+    private static AsyncProfiler profilerInstance;
+
+    static {
+        try {
+            String asyncProfilerLibPath = new 
File(ASYNC_PROFILER_LIB_PATH.getString()).getAbsolutePath(); //checkstyle: 
permit this instantiation

Review Comment:
   This OSX specific workaround might not be needed in newer versions of 
async-profiler, seems to have been fixed on 
https://github.com/async-profiler/async-profiler/issues/378
   
   Can you try with the latest async profiler version to check if this is 
needed? Thanks
   
   



##########
src/java/org/apache/cassandra/tools/nodetool/Profile.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.cassandra.tools.nodetool;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.profiler.AsyncProfilerMBean;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+@Command(name = "profile", description = "Run Async-Profiler on the cassandra 
process")
+public class Profile extends AbstractCommand {
+
+    @Option(names = {"-s", "--start"}, description = "Start profiling")
+    public boolean start;
+
+    @Option(names = {"-x", "--stop"}, description = "Stop profiling and dump 
output")
+    public boolean stop;
+
+    @Option(names = {"-e", "--event"}, description = "Event to profile (cpu, 
alloc, lock, wall, etc.)")
+    public String event = "cpu";
+
+    @Option(names = {"-r", "--raw"}, description = "Raw commands to execute")
+    public String raw;
+
+    @Option(names = {"-o", "--output"}, description = "Output file for profile 
dump")
+    public String outputFile = "/tmp/profile.html";
+
+    @Option(names = {"-f", "--format"}, description = "Output format 
(flamegraph, tree, traces, etc.)")
+    public String outputFormat = "flamegraph";
+
+    @Override
+    public void execute(NodeProbe probe) {
+        AsyncProfilerMBean profiler = probe.getAsyncProfilerProxy();
+        if (!profiler.isAvailable()) {
+            System.err.println("Async-profiler native library is not loaded or 
unavailable.");
+            return;
+        }
+
+        try {
+            if (start) {

Review Comment:
   ie.:
   ```java
           try
           {
               if (start)
               {
               }
           }
           catch (Exception e)
           {
              // handle
           }
   ```



##########
src/java/org/apache/cassandra/profiler/AsyncProfilerMBean.java:
##########
@@ -0,0 +1,8 @@
+package org.apache.cassandra.profiler;
+
+public interface AsyncProfilerMBean {
+    void start(String event, String outputFormat);

Review Comment:
   Instead of receiving only the `event` as parameter for `start`, I think it 
would be more extensible to receive a `Map<String,String> options` map, where 
`event` will be the initially supported 
[option](https://github.com/async-profiler/async-profiler/blob/master/docs/ProfilerOptions.md#options-applicable-to-any-output-format)
 in this map. Any other option should be rejected.
   
   In the future when we want to support more advanced options we don't need to 
change the interface.
   
   So this would become:
   
   ```java
   void start(Map<String, String> options);
   void stop();
   void executeUnsafe(String); // to emphasize this is "use at your own risk"
   boolean isAvailable();
   ```
   
   LMK what you think



##########
src/java/org/apache/cassandra/profiler/AsyncProfilerMBean.java:
##########
@@ -0,0 +1,8 @@
+package org.apache.cassandra.profiler;
+
+public interface AsyncProfilerMBean {
+    void start(String event, String outputFormat);
+    void stop(String outputFile);

Review Comment:
   In order to address this, we can have a 
`"cassandra.async_profiler.output_directory"` option and create a file in this 
directory, for example: `/tmp/cassandra-profiling/20252608T151934.html`.



##########
src/java/org/apache/cassandra/tools/profiler/AsyncProfilerService.java:
##########
@@ -0,0 +1,111 @@
+package org.apache.cassandra.tools.profiler;
+
+import one.profiler.AsyncProfiler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File; //checkstyle: permit this import
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_LIB_PATH;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_ENABLED;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.ASYNC_PROFILER_ADVANCED_MODE;
+
+public class AsyncProfilerService {
+    private static final Logger logger = 
LoggerFactory.getLogger(AsyncProfilerService.class);
+
+    private static final Set<String> VALID_EVENTS = Set.of("cpu", "alloc", 
"lock", "wall", "nativemem", "cache-misses");
+    private static final Set<String> VALID_FORMATS = 
Set.of("flat","traces","collapsed","flamegraph","tree","jfr","otlp");
+    private static final Character[] INVALID_OUTPUT_FILENAME_CHARS = {'"', 
'*', '<', '>', '?', '|'};
+
+    private static AsyncProfiler profilerInstance;
+
+    static {
+        try {
+            String asyncProfilerLibPath = new 
File(ASYNC_PROFILER_LIB_PATH.getString()).getAbsolutePath(); //checkstyle: 
permit this instantiation
+            profilerInstance = AsyncProfiler.getInstance(asyncProfilerLibPath);
+        } catch (Throwable t) {
+            System.out.println("async-profiler initialization ERROR");
+            t.printStackTrace();
+            profilerInstance = null;
+        }
+    }
+
+    public void start(String event, String outputFormat) {
+        checkProfilerInstance();
+        validateEvent(event);
+        validateFormat(outputFormat);
+
+        try {
+            String cmd = String.format("start,event=%s,output=%s", event, 
outputFormat);

Review Comment:
   In order to prevent an async profile session from running indefinitely let's 
add a default timeout of 60s and make that changeable via the `timeout` 
[option](https://github.com/async-profiler/async-profiler/blob/master/docs/ProfilerOptions.md#options-applicable-to-any-output-format):
   > The profiling duration, in seconds. The profiler will run for the 
specified period of time and then automatically stop.
   
   The user should still be able to stop a session with a timeout with via the 
`stop` method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to