Skip to content

Conversation

@pauloricardomg
Copy link
Contributor

Changes summary

Functionality

  • ProcessRuntimeConfiguration: represents the runtime configuration of a cassandra process, like environment variables, cassandra_home, cassandra_conf, etc. defined in sidecar.yaml
  • ProcessLifecycleProvider.start: constructs Cassandra runtime configuration from instance and global parameters and uses java ProcessBuilder API to start a Cassandra process writing a pidfile to the lifecycle state directory.
  • ProcessLifecycleProvider.stop: uses the pidfile to fetch process handle and send SIGKILL to stop the Cassandra process gracefully.
  • ProcessLifecycleProvider.isRunning: a Cassandra process is considered running if a PID file exists in the lifecycle state directory and a process with that PID is currently running.
  • InstanceMetadata and InstanceConfiguration: expose storageDir and instance-specific lifecycleOptions

Documentation

  • Add examples/lifecycle/README.md with an example on starting/stopping cassandra via lifecycle API
  • Update default sidecar.yaml with example and comments on how to enable ProcessLifecycleProvider

Infrastructure

  • Update build.gradle, integrationTestTask.gradle and gradle.properties to support running ProcessLifecycleProvider integration tests
  • Add scripts/build-cassandra-tarball.sh and update scripts/build-dtest-jars.sh` to build tarballs (requirement for running integration tests)
  • Update .circleci/config.yml to download cassandra tarballs for ProcessLifecycleProviderIntegrationTest and run integration tests for each supported version (4.0/5.0/5.1)

Testing

  • Add unit tests for ProcessLifecycleProvider to check runtime configuration is being constructed correctly and basic workflow (start/stop/status) with FakeLifecycleProvider
  • Add unit tests for ProcessRuntimeConfiguration to test start command construction and precondition validations
  • ProcessLifecycleProviderIntegrationTest: configures a local Cassandra install from a tarball in a temporary directory and run tests from LifecycleProviderIntegrationTester (start/stop/status)
    • TestFileUtils: contains utilities to install the cassandra tarball for ProcessLifecycleProviderIntegrationTest

Copy link
Contributor

@arjunashok arjunashok left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added some comments. Also, please add an entry for these changes into Changes.txt.

echo "Creating configuration"
cp -r ${CASSANDRA_HOME}/conf/* ${CASSANDRA_CONF}
sed "s#\$cassandraHome#${CASSANDRA_HOME}#g" ${SIDECAR_YAML_TEMPLATE} > ${SIDECAR_YAML}
sed -i "s#\$baseDir#${NODE_DIR}#g" ${SIDECAR_YAML}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was complaining on a Mac. The format that worked was -

sed -i '' "s#\$baseDir#${NODE_DIR}#g" ${SIDECAR_YAML}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, fixed on 9d08a6d

String stdoutLocation = getStdoutLocation(runtimeConfig.instanceName());
String stderrLocation = getStderrLocation(runtimeConfig.instanceName());
String pidFileLocation = getPidFileLocation(runtimeConfig.instanceName());
ProcessBuilder processBuilder = runtimeConfig.buildStartCommand(pidFileLocation,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like the additional files created here - stdout/stderr - are appended-to without rotation/cleanup and could result in disk space issues.

Also, there is a possibility of stale PID files if the process terminates abnormally. We would need some cleanup mechanism for such cases.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like the additional files created here - stdout/stderr - are appended-to without rotation/cleanup and could result in disk space issues.

These files are just startup logs, not the entire Cassandra logs which are logged to $CASSANDRA_LOG_DIR with proper rotation. In order to not grow the scope of this PR while keeping the file sizes bounded I updated this patch to overwrite the pervious startup logs instead of appending indefinitely on c1eb011. If needed we can add append+rotation support later.

Also, there is a possibility of stale PID files if the process terminates abnormally. We would need some cleanup mechanism for such cases.

Good call. I've enhanced ProcessLifecycleProvider.isCassandraProcessRunning to auto-remove the PID file when the PID file exists but no Cassandra process is running, as well ensure the PID file is removed on ProcessLifecycleProvider.stop (0b65e3e).

LOG.info("Starting Cassandra instance {} with command: {}", runtimeConfig.instanceName(), processBuilder.command());

Process process = processBuilder.start();
process.waitFor(CASSANDRA_PROCESS_TIMEOUT_MS, TimeUnit.MILLISECONDS); // blocking call, make async?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As you have called out, consider using some sort of a periodic check for process, Instead of blocking the thread. Likewise for the stop process below.

Suggested change
process.waitFor(CASSANDRA_PROCESS_TIMEOUT_MS, TimeUnit.MILLISECONDS); // blocking call, make async?
vertx.setPeriodic(1000, timerId -> {
if (isCassandraProcessRunning(instance)) {
vertx.cancelTimer(timerId);
LOG.info("Started Cassandra instance {} with PID {}", runtimeConfig.instanceName(),
readPidFromFile(Path.of(pidFileLocation)));
} else if (System.currentTimeMillis() > startTime + CASSANDRA_PROCESS_TIMEOUT_MS) {
vertx.cancelTimer(timerId);
LOG.error("Failed to start Cassandra instance {} within timeout", runtimeConfig.instanceName());
// Handle timeout error
}
});

Copy link
Contributor Author

@pauloricardomg pauloricardomg Dec 5, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My understanding is that vertx.setPeriodic will return immediatelly right ? In which case, LifecycleProvider.start could return before the Cassandra process is initialized.

The issue is that LifecycleManager.submitStartTask assumes LifecycleProvider.start is synchronous and will only return after the process is guaranteed to be started. Are you suggesting making LifecycleProvider.start asynchronous and return a Future instead? Since this is a departure from the current synchronous design and may require many changes I would prefer to do this in a follow-up ticket if that works for you.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep, valid point about the architectural impact of going fully async. Follow-up ticket makes sense.

I was hoping to move away from blocking the worker thread from waiting on a kernel system call, towards a Java synchronization primitive (like CountDownLatch), which is more efficient and interruptible. Though I recognize this would be an intermediate step - if we're planning to eventually adopt an async interface. We'd need to weigh whether this intermediate improvement justifies the additional churn.

Additionally process.waitFor(timeout) returns false when it times out, but we don't check the return value. Wondering if it is worth detecting the status of the process start for better failure handling.

{
LOG.info("Stopping process of Cassandra instance {} with PID {}.", casCfg.instanceName(), pid);
CompletableFuture<ProcessHandle> terminationFuture = processHandle.get().onExit();
processHandle.get().destroy(); // blocking call, make async?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider a force destroy destroyForcibly() on timeout with appropriate logging.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done on 56ca1f6

Comment on lines +70 to +75
this.lifecycleDir = params.get(OPT_STATE_DIR);
this.defaultCassandraHome = params.get(OPT_CASSANDRA_HOME);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we fail-fast if these directories do not exist or do not have appropriate permissions?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed on 369363a

@pauloricardomg pauloricardomg force-pushed the CASSSIDECAR-340 branch 4 times, most recently from 995176a to e025874 Compare December 5, 2025 23:39
Copy link
Contributor

@arjunashok arjunashok left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for addressing the comments. Looks good. Added a comment to fix a spot bugs failure.

Please add an entry to Changes.txt, and share a link to the CI build.

ProcessBuilder pb = new ProcessBuilder("ps", "-p", String.valueOf(pid), "-o", "args=");
Process proc = pb.start();
try (BufferedReader reader = new BufferedReader(
new InputStreamReader(proc.getInputStream())))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To address the spotbugs error

Suggested change
new InputStreamReader(proc.getInputStream())))
new InputStreamReader(proc.getInputStream(), StandardCharsets.UTF_8)))

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated on b643899

@pauloricardomg pauloricardomg force-pushed the CASSSIDECAR-340 branch 4 times, most recently from b071a11 to d7d8098 Compare December 22, 2025 19:17
pauloricardomg and others added 12 commits December 23, 2025 11:42
Empty string argument to -i works on both BSD and GNU sed.
- Verify process is Cassandra before considering it running
- Auto-remove stale PID files when process missing or not Cassandra
- Add cleanup in stopCassandra() and improved process validation
- Add unit tests for stale PID scenarios
Co-authored-by: Arjun Ashok <arjun.diablo@gmail.com>
When stopping a Cassandra process, if the graceful destroy() does not
complete within the timeout period, the code now calls destroyForcibly()
to ensure the process is terminated. If destroyForcibly() also fails
(returns false), an exception is thrown.
Validates that state directory and cassandra home exist and have
appropriate permissions at construction time.
Configure -PtarballVersion for heavyweight integration tests to fix
ProcessLifecycleProviderIntegrationTest which requires cassandra.test.tarball_path.
The sidecar.version resource file is generated during the jar task.
Without running jar first, SidecarVersionProvider throws
IllegalStateException when trying to read the missing resource.

This aligns GitHub Actions with CircleCI which already includes jar
before integrationTestHeavyWeight.
@pauloricardomg
Copy link
Contributor Author

Thanks for the review @arjunashok - updated github actions to support new intergration tests for lifecycle provider. All the checks have passed.

Copy link
Contributor

@arjunashok arjunashok left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, +1. Thanks for addressing the comments.

Couple of nits:

  • Consider addressing the comment here to handle timeout return value.
  • Add entry to Changes.txt

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants