Skip to content

Commit

Permalink
Encapsulate jmx interactions in a proxy class
Browse files Browse the repository at this point in the history
This allows us to hide the logic behind connecting to JMX and better handle the sidecar mode.
Notably, it will hide the choices between connecting directly through JMX, getting info from the backend or using an async request/response pattern.
JMX communication changes for the sidecar mode
I added a set of accessible datacenters that are useful to determine if a node is accessible from a Reaper instance in EACH mode notably. The set is populated during the JMX connection initialization phase where we try to connect to all nodes.
The active compactions are now polled and stored into the backend every 10 seconds so they can be retrieved from any reaper in sidecar mode.
add a guava cache inside `EndpointSnitchInfoProxy.getDataCenter(host)` as it's a static and low-cardinality lookup
  • Loading branch information
adejanovski committed Feb 28, 2019
1 parent d001c47 commit f6c451b
Show file tree
Hide file tree
Showing 42 changed files with 1,887 additions and 820 deletions.
2 changes: 1 addition & 1 deletion src/server/src/checkstyle/java.header
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
/*
( * Copyright 2014-2017 Spotify AB)?
( * Copyright \d\d\d\d-2018 The Last Pickle Ltd)?
( * Copyright \d\d\d\d-\d\d\d\d The Last Pickle Ltd)?
*
* Licensed under the Apache License, Version 2.0 \(the "License"\);
* you may not use this file except in compliance with the License.
Expand Down
23 changes: 21 additions & 2 deletions src/server/src/main/java/io/cassandrareaper/ReaperApplication.java
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ public void run(ReaperApplicationConfiguration config, Environment environment)

if (context.jmxConnectionFactory == null) {
LOG.info("no JMX connection factory given in context, creating default");
context.jmxConnectionFactory = new JmxConnectionFactory(context.metricRegistry);
context.jmxConnectionFactory = new JmxConnectionFactory(context);

// read jmx host/port mapping from config and provide to jmx con.factory
Map<String, Integer> jmxPorts = config.getJmxPorts();
Expand Down Expand Up @@ -266,14 +266,15 @@ public void run(ReaperApplicationConfiguration config, Environment environment)
"Cassandra backend storage is the only one allowing EACH datacenter availability modes.");

ScheduledExecutorService scheduler = new InstrumentedScheduledExecutorService(
environment.lifecycle().scheduledExecutorService("ReaperApplication-scheduler").threads(1).build(),
environment.lifecycle().scheduledExecutorService("ReaperApplication-scheduler").threads(3).build(),
context.metricRegistry);

if (context.storage instanceof IDistributedStorage) {
// Allowing multiple Reaper instances to work concurrently requires
// us to poll the database for running repairs regularly
// only with Cassandra storage
scheduleRepairManager(scheduler);
scheduleHandleMetricsRequest(scheduler);
} else {
// Storage is different than Cassandra, assuming we have a single instance
context.repairManager.resumeRunningRepairRuns();
Expand All @@ -285,6 +286,7 @@ public void run(ReaperApplicationConfiguration config, Environment environment)
LOG.warn("Reaper is ready to get things done!");
}


private void scheduleRepairManager(ScheduledExecutorService scheduler) {
scheduler.scheduleWithFixedDelay(
() -> {
Expand All @@ -302,6 +304,23 @@ private void scheduleRepairManager(ScheduledExecutorService scheduler) {
TimeUnit.SECONDS);
}

private void scheduleHandleMetricsRequest(ScheduledExecutorService scheduler) {
scheduler.scheduleWithFixedDelay(
() -> {
try {
context.repairManager.handleMetricsRequests();
} catch (ReaperException | RuntimeException e) {
// test-pollution: grim_reaper trashes this log error
//if (!Boolean.getBoolean("grim.reaper.running")) {
LOG.error("Couldn't handle metrics requests", e);
//}
}
},
0,
10,
TimeUnit.SECONDS);
}

private void schedulePurge(ScheduledExecutorService scheduler) {
final PurgeService purgeManager = PurgeService.create(context);

Expand Down
152 changes: 152 additions & 0 deletions src/server/src/main/java/io/cassandrareaper/core/GenericMetric.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
*
* Copyright 2019-2019 The Last Pickle Ltd
*
* Licensed 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 io.cassandrareaper.core;

import org.joda.time.DateTime;

public final class GenericMetric {

private final String clusterName;
private final String metricDomain;
private final String metricType;
private final String metricScope;
private final String metricName;
private final String metricAttribute;
private final String host;
private final DateTime ts;
private final double value;

private GenericMetric(Builder builder) {
this.clusterName = builder.clusterName;
this.metricDomain = builder.metricDomain;
this.metricType = builder.metricType;
this.metricScope = builder.metricScope;
this.metricName = builder.metricName;
this.metricAttribute = builder.metricAttribute;
this.host = builder.host;
this.ts = builder.ts;
this.value = builder.value;
}

public String getClusterName() {
return clusterName;
}

public String getMetricDomain() {
return metricDomain;
}

public String getMetricType() {
return metricType;
}

public String getMetricScope() {
return metricScope;
}

public String getMetricName() {
return metricName;
}

public String getMetricAttribute() {
return metricAttribute;
}

public String getMetricFullId() {
return metricDomain + ":type=" + metricType + ",scope=" + metricScope + ",name=" + metricName;
}

public String getHost() {
return host;
}

public DateTime getTs() {
return ts;
}

public double getValue() {
return value;
}

public static Builder builder() {
return new Builder();
}

public static final class Builder {
private String clusterName;
private String metricDomain;
private String metricType;
private String metricScope;
private String metricName;
private String metricAttribute;
private String host;
private DateTime ts;
private double value;

private Builder() {}

public Builder withClusterName(String clusterName) {
this.clusterName = clusterName;
return this;
}

public Builder withMetricDomain(String metricDomain) {
this.metricDomain = metricDomain;
return this;
}

public Builder withMetricType(String metricType) {
this.metricType = metricType;
return this;
}

public Builder withMetricScope(String metricScope) {
this.metricScope = metricScope == null ? " " : metricScope;
return this;
}

public Builder withMetricName(String metricName) {
this.metricName = metricName;
return this;
}

public Builder withMetricAttribute(String metricAttribute) {
this.metricAttribute = metricAttribute;
return this;
}

public Builder withHost(String host) {
this.host = host;
return this;
}

public Builder withTs(DateTime ts) {
this.ts = ts;
return this;
}

public Builder withValue(double value) {
this.value = value;
return this;
}

public GenericMetric build() {
return new GenericMetric(this);
}
}
}
38 changes: 37 additions & 1 deletion src/server/src/main/java/io/cassandrareaper/core/JmxStat.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,18 +22,36 @@

@JsonDeserialize(builder = JmxStat.Builder.class)
public final class JmxStat {
private final String mbeanName;
private final String domain;
private final String type;
private final String scope;
private final String name;
private final String attribute;
private final Double value;

private JmxStat(Builder builder) {
this.mbeanName = builder.mbeanName;
this.domain = builder.domain;
this.type = builder.type;
this.scope = builder.scope;
this.name = builder.name;
this.attribute = builder.attribute;
this.value = builder.value;
}

public String getMbeanName() {
return mbeanName;
}

public String getDomain() {
return domain;
}

public String getType() {
return type;
}

public String getScope() {
return scope;
}
Expand All @@ -53,7 +71,7 @@ public Double getValue() {

@Override
public String toString() {
return scope + "/" + name + "/" + attribute + " = " + value;
return mbeanName + "/" + scope + "/" + name + "/" + attribute + " = " + value;
}

public static Builder builder() {
Expand All @@ -62,13 +80,31 @@ public static Builder builder() {

@JsonPOJOBuilder(buildMethodName = "build", withPrefix = "with")
public static final class Builder {
private String mbeanName;
private String domain;
private String type;
private String scope;
private String name;
private String attribute;
private Double value;

private Builder() {}

public Builder withMbeanName(String mbeanName) {
this.mbeanName = mbeanName;
return this;
}

public Builder withDomain(String domain) {
this.domain = domain;
return this;
}

public Builder withType(String type) {
this.type = type;
return this;
}

public Builder withScope(String scope) {
this.scope = scope;
return this;
Expand Down
Loading

0 comments on commit f6c451b

Please sign in to comment.