[ https://issues.apache.org/jira/browse/CASSANDRA-4479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Chris Merrill updated CASSANDRA-4479: ------------------------------------- Attachment: trunk-4479.txt > JMX attribute setters not consistent with cassandra.yaml > -------------------------------------------------------- > > Key: CASSANDRA-4479 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4479 > Project: Cassandra > Issue Type: Bug > Affects Versions: 1.1.2 > Reporter: Eric Dong > Attachments: trunk-4479.txt > > > If a setting is configurable both via cassandra.yaml and JMX, the two should > be consistent. If that doesn't hold, then the JMX setter can't be trusted. > Here I present the example of phi_convict_threshold. > I'm trying to set phi_convict_threshold via JMX, which sets > FailureDetector.phiConvictThreshold_, but this doesn't update > Config.phi_convict_threshold, which gets its value from cassandra.yaml when > starting up. > Some places, such as FailureDetector.interpret(InetAddress), use > FailureDetector.phiConvictThreshold_; others, such as AntiEntropyService.line > 813 in cassandra-1.1.2, use Config.phi_convict_threshold: > {code} > // We want a higher confidence in the failure detection than > usual because failing a repair wrongly has a high cost. > if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold()) > return; > {code} > where DatabaseDescriptor.getPhiConvictThreshold() returns > Conf.phi_convict_threshold. > So, it looks like there are cases where a value is stored in multiple places, > and setting the value via JMX doesn't set all of them. I'd say there should > only be a single place where a configuration parameter is stored, and that > single field: > * should read in the value from cassandra.yaml, optionally falling back to a > sane default > * should be the field that the JMX attribute reads and sets, and > * any place that needs the current global setting should get it from that > field. However, there could be cases where you read in a global value at the > start of a task and keep that value locally until the end of the task. > Also, anything settable via JMX should be volatile or set via a synchronized > setter, or else according to the Java memory model other threads may be stuck > with the old setting. > So, I'm requesting the following: > * Setting up guidelines for how to expose a configuration parameter both via > cassandra.yaml and JMX, based on what I've mentioned above > * Going through the list of configuration parameters and fixing any that > don't match those guidelines > I'd also recommend logging any changes to configuration parameters. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira