diff --git a/common/pom.xml b/common/pom.xml index 2267c33..466a424 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -14,7 +14,7 @@ Cassandra Exporter Common - 3.6.1 + 3.9.5 diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/FactoriesSupplier.java b/common/src/main/java/com/zegelin/cassandra/exporter/FactoriesSupplier.java index 2d88503..caab14a 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/FactoriesSupplier.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/FactoriesSupplier.java @@ -43,7 +43,7 @@ private static class FactoryBuilder { interface Modifier { /** * @param keyPropertyList Map of MBean ObjectName key properties and their values. - * @param labels The current map of labels to be provided to the collector constructor. + * @param labels The current map of labels to be provided to the collector constructor. * @return true to continue building the collector, false to abort. */ boolean modify(final Map keyPropertyList, final Map labels); @@ -72,7 +72,7 @@ FactoryBuilder withLabelMaker(final LabelMaker labelMaker) { return this.withModifier((keyPropertyList, labels) -> { labels.putAll(labelMaker.apply(keyPropertyList)); return true; - }); + }); } FactoryBuilder withHelp(final String help) { @@ -116,6 +116,7 @@ public interface CollectorConstructor { private final Set tableLabels; private final Set excludedKeyspaces; private final Map tableMetricScopeFilters; + private final Set exclusions; public FactoriesSupplier(final MetadataFactory metadataFactory, final HarvesterOptions options) { @@ -129,6 +130,7 @@ public FactoriesSupplier(final MetadataFactory metadataFactory, final HarvesterO .put(TableMetricScope.KEYSPACE, options.keyspaceMetricsFilter) .put(TableMetricScope.TABLE, options.tableMetricsFilter) .build(); + this.exclusions = options.exclusions; } @@ -516,14 +518,12 @@ private static FactoryBuilder.CollectorConstructor functionalCollectorConstr } - - private Factory cache(final Factory delegate, final long duration, final TimeUnit unit) { return CachingCollector.cache(delegate, duration, unit); } private Iterator cache(final Iterator delegates, final long duration, final TimeUnit unit) { - return Iterators.transform(delegates, delegate -> CachingCollector.cache(delegate, duration, unit)); + return Iterators.transform(delegates, delegate -> CachingCollector.cache(delegate, duration, unit)); } @@ -532,10 +532,10 @@ public List get() { final ImmutableList.Builder builder = ImmutableList.builder(); builder.add(FailureDetectorMBeanMetricFamilyCollector.factory(metadataFactory)); - builder.add(cache(StorageServiceMBeanMetricFamilyCollector.factory(metadataFactory, excludedKeyspaces), 5, TimeUnit.MINUTES)); + builder.add(cache(StorageServiceMBeanMetricFamilyCollector.factory(metadataFactory, excludedKeyspaces, exclusions), 5, TimeUnit.MINUTES)); builder.add(MemoryPoolMXBeanMetricFamilyCollector.FACTORY); - builder.add(GarbageCollectorMXBeanMetricFamilyCollector.FACTORY); + builder.add(GarbageCollectorMXBeanMetricFamilyCollector.factory(exclusions)); builder.add(BufferPoolMXBeanMetricFamilyCollector.FACTORY); builder.add(cache(OperatingSystemMXBeanMetricFamilyCollector.FACTORY, 5, TimeUnit.MINUTES)); builder.add(ThreadMXBeanMetricFamilyCollector.factory(perThreadTimingEnabled)); @@ -635,7 +635,7 @@ public List get() { // org.apache.cassandra.metrics.CompactionMetrics { - builder.add(compactionMetric(functionalCollectorConstructor(counterAsCounter()),"BytesCompacted", "bytes_compacted_total", "Total number of bytes compacted (since server start).")); + builder.add(compactionMetric(functionalCollectorConstructor(counterAsCounter()), "BytesCompacted", "bytes_compacted_total", "Total number of bytes compacted (since server start).")); builder.add(compactionMetric(functionalCollectorConstructor(numericGaugeAsCounter()), "CompletedTasks", "completed_tasks_total", "Total number of completed compaction tasks (since server start).")); // "PendingTasks" ignored -- it's an aggregate of the table-level metrics (see the table metric "PendingCompactions") builder.add(compactionMetric(functionalCollectorConstructor(meterAsCounter()), "TotalCompactionsCompleted", "completed_total", "Total number of compactions (since server start).")); diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/Harvester.java b/common/src/main/java/com/zegelin/cassandra/exporter/Harvester.java index ae433c9..0a86995 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/Harvester.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/Harvester.java @@ -7,6 +7,7 @@ import com.zegelin.jmx.NamedObject; import com.zegelin.cassandra.exporter.cli.HarvesterOptions; import com.zegelin.prometheus.domain.CounterMetricFamily; +import com.zegelin.prometheus.domain.Interval.Quantile; import com.zegelin.prometheus.domain.Labels; import com.zegelin.prometheus.domain.MetricFamily; import com.zegelin.prometheus.domain.NumericMetric; @@ -132,6 +133,7 @@ public boolean equals(final Object o) { private final boolean collectorTimingEnabled; private final Map collectionTimes = new ConcurrentHashMap<>(); + private final Set excludedHistoQuantiles; private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() .setNameFormat("cassandra-exporter-harvester-defer-%d") @@ -145,6 +147,11 @@ protected Harvester(final MetadataFactory metadataFactory, final HarvesterOption this.exclusions = options.exclusions; this.enabledGlobalLabels = options.globalLabels; this.collectorTimingEnabled = options.collectorTimingEnabled; + this.excludedHistoQuantiles = options.excludedHistoQuantiles; + } + + public Set getExcludedHistoQuantiles() { + return excludedHistoQuantiles; } protected void addCollectorFactory(final MBeanGroupMetricFamilyCollector.Factory factory) { diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/cli/HarvesterOptions.java b/common/src/main/java/com/zegelin/cassandra/exporter/cli/HarvesterOptions.java index dea9e82..1b228e0 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/cli/HarvesterOptions.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/cli/HarvesterOptions.java @@ -2,6 +2,7 @@ import com.google.common.collect.ImmutableSet; import com.zegelin.netty.Floats; +import com.zegelin.prometheus.domain.Interval.Quantile; import com.zegelin.cassandra.exporter.FactoriesSupplier; import com.zegelin.cassandra.exporter.Harvester; import picocli.CommandLine; @@ -152,4 +153,27 @@ public void setExcludeSystemTables(final boolean excludeSystemTables) { excludedKeyspaces.addAll(CASSANDRA_SYSTEM_KEYSPACES); } + + public final Set excludedHistoQuantiles = new HashSet<>(); + @Option(names = {"--exclude-from-histogram"}, paramLabel = "EXCLUSION", arity = "1..*", + description = "Select which quantiles to exclude from histogram metrics. The specified quantiles are excluded from all histogram/summary metrics" + + "Valid options are: P_50, P_75, P_95, P_98, P_99, P_99_9" + + "'P_50' (Quantile .5), " + + "'P_75' (Quantile .75), " + + "'P_95' (Quantile .95), " + + "'P_98' (Quantile .98). " + + "'P_99' (Quantile .99). " + + "'P_99_9' (Quantile .999). " + + "The default is to include all quantiles. " + ) + void setExcludeFromHistogram(final Set values) { + values.forEach( e -> { + Quantile q = Quantile.ALL_PERCENTILES.get(e); + if(q == null) { + throw new IllegalArgumentException(String.format("The specified exlusion quantile '%s' is invalid, value values are '%s'", e, Quantile.ALL_PERCENTILES.keySet())); + } + excludedHistoQuantiles.add(q); + }); + } + } diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/collector/StorageServiceMBeanMetricFamilyCollector.java b/common/src/main/java/com/zegelin/cassandra/exporter/collector/StorageServiceMBeanMetricFamilyCollector.java index f7cc897..792586e 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/collector/StorageServiceMBeanMetricFamilyCollector.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/collector/StorageServiceMBeanMetricFamilyCollector.java @@ -2,6 +2,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.zegelin.cassandra.exporter.Harvester; import com.zegelin.cassandra.exporter.MBeanGroupMetricFamilyCollector; import com.zegelin.cassandra.exporter.MetadataFactory; import com.zegelin.prometheus.domain.GaugeMetricFamily; @@ -26,13 +27,14 @@ public class StorageServiceMBeanMetricFamilyCollector extends MBeanGroupMetricFamilyCollector { private static final Logger logger = LoggerFactory.getLogger(StorageServiceMBeanMetricFamilyCollector.class); + private final Set exclusions; - public static Factory factory(final MetadataFactory metadataFactory, final Set excludedKeyspaces) { + public static Factory factory(final MetadataFactory metadataFactory, final Set excludedKeyspaces, final Set exclusions) { return mBean -> { if (!STORAGE_SERVICE_MBEAN_NAME.apply(mBean.name)) return null; - return new StorageServiceMBeanMetricFamilyCollector((StorageServiceMBean) mBean.object, metadataFactory, excludedKeyspaces); + return new StorageServiceMBeanMetricFamilyCollector((StorageServiceMBean) mBean.object, metadataFactory, excludedKeyspaces, exclusions); }; } @@ -45,10 +47,11 @@ public static Factory factory(final MetadataFactory metadataFactory, final Set excludedKeyspaces) { + final MetadataFactory metadataFactory, final Set excludedKeyspaces, final Set exclusions) { this.storageServiceMBean = storageServiceMBean; this.metadataFactory = metadataFactory; this.excludedKeyspaces = excludedKeyspaces; + this.exclusions=exclusions; // determine the set of FileStores (i.e., mountpoints) for the Cassandra data/CL/cache directories // (which can be done once -- changing directories requires a server restart) @@ -144,6 +147,6 @@ public Stream collect() { metricFamilyStreamBuilder.add(new GaugeMetricFamily("cassandra_storage_filesystem_unallocated_bytes", null, fileStoreUnallocatedSpaceMetrics.build())); } - return metricFamilyStreamBuilder.build(); + return metricFamilyStreamBuilder.build().filter(mf -> exclusions.contains(mf.name)); } } diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/collector/jvm/GarbageCollectorMXBeanMetricFamilyCollector.java b/common/src/main/java/com/zegelin/cassandra/exporter/collector/jvm/GarbageCollectorMXBeanMetricFamilyCollector.java index 7d5b3ac..6580cc1 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/collector/jvm/GarbageCollectorMXBeanMetricFamilyCollector.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/collector/jvm/GarbageCollectorMXBeanMetricFamilyCollector.java @@ -2,6 +2,7 @@ import com.google.common.collect.ImmutableMap; import com.sun.management.GcInfo; +import com.zegelin.cassandra.exporter.Harvester; import com.zegelin.jmx.ObjectNames; import com.zegelin.cassandra.exporter.MBeanGroupMetricFamilyCollector; import com.zegelin.prometheus.domain.*; @@ -11,6 +12,7 @@ import java.lang.management.ManagementFactory; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.stream.Stream; import static com.zegelin.cassandra.exporter.MetricValueConversionFunctions.millisecondsToSeconds; @@ -18,21 +20,25 @@ public class GarbageCollectorMXBeanMetricFamilyCollector extends MBeanGroupMetricFamilyCollector { private static final ObjectName GARBAGE_COLLECTOR_MXBEAN_NAME_PATTERN = ObjectNames.create(ManagementFactory.GARBAGE_COLLECTOR_MXBEAN_DOMAIN_TYPE + ",*"); + private final Set exclusions; - public static final Factory FACTORY = mBean -> { - if (!GARBAGE_COLLECTOR_MXBEAN_NAME_PATTERN.apply(mBean.name)) - return null; + public static Factory factory(final Set exclusions) { + return mBean -> { + if (!GARBAGE_COLLECTOR_MXBEAN_NAME_PATTERN.apply(mBean.name)) + return null; - final GarbageCollectorMXBean garbageCollectorMXBean = (GarbageCollectorMXBean) mBean.object; + final GarbageCollectorMXBean garbageCollectorMXBean = (GarbageCollectorMXBean) mBean.object; - final Labels collectorLabels = Labels.of("collector", garbageCollectorMXBean.getName()); + final Labels collectorLabels = Labels.of("collector", garbageCollectorMXBean.getName()); - return new GarbageCollectorMXBeanMetricFamilyCollector(ImmutableMap.of(collectorLabels, garbageCollectorMXBean)); - }; + return new GarbageCollectorMXBeanMetricFamilyCollector(ImmutableMap.of(collectorLabels, garbageCollectorMXBean), exclusions); + }; + } private final Map labeledGarbageCollectorMXBeans; - private GarbageCollectorMXBeanMetricFamilyCollector(final Map labeledGarbageCollectorMXBeans) { + private GarbageCollectorMXBeanMetricFamilyCollector(final Map labeledGarbageCollectorMXBeans, Set exclusions) { + this.exclusions = exclusions; this.labeledGarbageCollectorMXBeans = labeledGarbageCollectorMXBeans; } @@ -46,10 +52,12 @@ public MBeanGroupMetricFamilyCollector merge(final MBeanGroupMetricFamilyCollect final Map labeledGarbageCollectorMXBeans = new HashMap<>(this.labeledGarbageCollectorMXBeans); for (final Map.Entry entry : other.labeledGarbageCollectorMXBeans.entrySet()) { - labeledGarbageCollectorMXBeans.merge(entry.getKey(), entry.getValue(), (o1, o2) -> {throw new IllegalStateException(String.format("Object %s and %s cannot be merged, yet their labels are the same.", o1, o2));}); + labeledGarbageCollectorMXBeans.merge(entry.getKey(), entry.getValue(), (o1, o2) -> { + throw new IllegalStateException(String.format("Object %s and %s cannot be merged, yet their labels are the same.", o1, o2)); + }); } - return new GarbageCollectorMXBeanMetricFamilyCollector(labeledGarbageCollectorMXBeans); + return new GarbageCollectorMXBeanMetricFamilyCollector(labeledGarbageCollectorMXBeans, exclusions); } @Override @@ -73,11 +81,10 @@ public Stream collect() { } } } - - return Stream.of( - new CounterMetricFamily("cassandra_jvm_gc_collection_count", "Total number of collections that have occurred (since JVM start).", collectionCountMetrics.build()), - new CounterMetricFamily("cassandra_jvm_gc_estimated_collection_duration_seconds_total", "Estimated cumulative collection elapsed time (since JVM start).", collectionDurationTotalSecondsMetrics.build()), - new GaugeMetricFamily("cassandra_jvm_gc_last_collection_duration_seconds", "Last collection duration.", lastGCDurationSecondsMetrics.build()) - ); + final Stream.Builder metricFamilyStreamBuilder = Stream.builder(); + metricFamilyStreamBuilder.add(new CounterMetricFamily("cassandra_jvm_gc_collection_count", "Total number of collections that have occurred (since JVM start).", collectionCountMetrics.build())); + metricFamilyStreamBuilder.add(new CounterMetricFamily("cassandra_jvm_gc_estimated_collection_duration_seconds_total", "Estimated cumulative collection elapsed time (since JVM start).", collectionDurationTotalSecondsMetrics.build())); + metricFamilyStreamBuilder.add(new GaugeMetricFamily("cassandra_jvm_gc_last_collection_duration_seconds", "Last collection duration.", lastGCDurationSecondsMetrics.build())); + return metricFamilyStreamBuilder.build().filter(mf -> exclusions.contains(mf.name)); } } diff --git a/common/src/main/java/com/zegelin/cassandra/exporter/netty/HttpHandler.java b/common/src/main/java/com/zegelin/cassandra/exporter/netty/HttpHandler.java index e065f1d..02c9ca9 100644 --- a/common/src/main/java/com/zegelin/cassandra/exporter/netty/HttpHandler.java +++ b/common/src/main/java/com/zegelin/cassandra/exporter/netty/HttpHandler.java @@ -299,7 +299,7 @@ private ChannelFuture sendMetrics(final ChannelHandlerContext ctx, final FullHtt lastWriteFuture = ctx.writeAndFlush(response); if (request.getMethod() == HttpMethod.GET) { - ReadableByteChannel byteChannel = new FormattedByteChannel(new TextFormatExposition(metricFamilyStream, timestamp, globalLabels, includeHelp)); + ReadableByteChannel byteChannel = new FormattedByteChannel(new TextFormatExposition(metricFamilyStream, timestamp, globalLabels, includeHelp, harvester.getExcludedHistoQuantiles())); lastWriteFuture = ctx.writeAndFlush(new HttpChunkedInput(new ChunkedNioStream(byteChannel, FormattedByteChannel.MAX_CHUNK_SIZE))); } diff --git a/common/src/main/java/com/zegelin/prometheus/domain/Interval.java b/common/src/main/java/com/zegelin/prometheus/domain/Interval.java index 1394ded..ec9b191 100644 --- a/common/src/main/java/com/zegelin/prometheus/domain/Interval.java +++ b/common/src/main/java/com/zegelin/prometheus/domain/Interval.java @@ -1,13 +1,13 @@ package com.zegelin.prometheus.domain; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.zegelin.function.FloatFloatFunction; +import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; /* A Summary quanitle or Histogram bucket and associated value. @@ -23,6 +23,14 @@ public static class Quantile { public static final Set STANDARD_PERCENTILES = ImmutableSet.of(P_50, P_75, P_95, P_98, P_99, P_99_9); public static final Quantile POSITIVE_INFINITY = q(Float.POSITIVE_INFINITY); + public static final Map ALL_PERCENTILES = new ImmutableMap.Builder() + .put("P_50",P_50) + .put("P_75",P_75) + .put("P_95",P_95) + .put("P_98",P_98) + .put("P_99",P_99) + .put("P_99_9",P_99_9) + .build(); public final float value; diff --git a/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatExposition.java b/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatExposition.java index 4c53c6f..bd4040f 100644 --- a/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatExposition.java +++ b/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatExposition.java @@ -3,12 +3,14 @@ import com.google.common.base.Stopwatch; import com.zegelin.netty.Resources; import com.zegelin.prometheus.domain.*; +import com.zegelin.prometheus.domain.Interval.Quantile; import com.zegelin.prometheus.exposition.ExpositionSink; import com.zegelin.prometheus.exposition.FormattedExposition; import io.netty.buffer.ByteBuf; import java.time.Instant; import java.util.Iterator; +import java.util.Set; import java.util.stream.Stream; public class TextFormatExposition implements FormattedExposition { @@ -35,13 +37,14 @@ private enum State { private int metricCount = 0; private final Stopwatch stopwatch = Stopwatch.createUnstarted(); - - - public TextFormatExposition(final Stream metricFamilies, final Instant timestamp, final Labels globalLabels, final boolean includeHelp) { + private final Set excludedHistoQuantiles; + + public TextFormatExposition(final Stream metricFamilies, final Instant timestamp, final Labels globalLabels, final boolean includeHelp, final Set excludedHistoQuantiles) { this.metricFamiliesIterator = metricFamilies.iterator(); this.timestamp = timestamp; this.globalLabels = globalLabels; this.includeHelp = includeHelp; + this.excludedHistoQuantiles = excludedHistoQuantiles; } @Override @@ -70,7 +73,7 @@ public void nextSlice(final ExpositionSink chunkBuffer) { final MetricFamily metricFamily = metricFamiliesIterator.next(); - metricFamilyWriter = new TextFormatMetricFamilyWriter(timestamp, globalLabels, includeHelp, metricFamily); + metricFamilyWriter = new TextFormatMetricFamilyWriter(timestamp, globalLabels, includeHelp, metricFamily, excludedHistoQuantiles); metricFamilyWriter.writeFamilyHeader(chunkBuffer); diff --git a/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatMetricFamilyWriter.java b/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatMetricFamilyWriter.java index e0209f7..017f01b 100644 --- a/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatMetricFamilyWriter.java +++ b/common/src/main/java/com/zegelin/prometheus/exposition/text/TextFormatMetricFamilyWriter.java @@ -3,10 +3,12 @@ import com.google.common.escape.CharEscaperBuilder; import com.google.common.escape.Escaper; import com.zegelin.prometheus.domain.*; +import com.zegelin.prometheus.domain.Interval.Quantile; import com.zegelin.prometheus.exposition.ExpositionSink; import java.time.Instant; import java.util.Iterator; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -41,14 +43,16 @@ void write(final ExpositionSink buffer) { private final Consumer> headerWriter; private final Function, Boolean> metricWriter; + private final Set excludedHistoQuantiles; - TextFormatMetricFamilyWriter(final Instant timestamp, final Labels globalLabels, final boolean includeHelp, final MetricFamily metricFamily) { + TextFormatMetricFamilyWriter(final Instant timestamp, final Labels globalLabels, final boolean includeHelp, final MetricFamily metricFamily, final Set excludedHistoQuantiles) { this.timestamp = " " + timestamp.toEpochMilli(); this.globalLabels = globalLabels; this.includeHelp = includeHelp; this.headerWriter = metricFamily.accept(new HeaderVisitor()); this.metricWriter = metricFamily.accept(new MetricVisitor()); + this.excludedHistoQuantiles=excludedHistoQuantiles; } class HeaderVisitor implements MetricFamilyVisitor>> { @@ -182,7 +186,9 @@ public Function, Boolean> visit(final SummaryMetricFamily metr writeMetric(buffer, metricFamily, "_count", summary.count, summary.labels); summary.quantiles.forEach(interval -> { - writeMetric(buffer, metricFamily, null, interval.value, summary.labels, interval.quantile.asSummaryLabel()); + if (!excludedHistoQuantiles.contains(interval.quantile)) { + writeMetric(buffer, metricFamily, null, interval.value, summary.labels, interval.quantile.asSummaryLabel()); + } }); }); } @@ -194,7 +200,10 @@ public Function, Boolean> visit(final HistogramMetricFamily me writeMetric(buffer, metricFamily, "_count", histogram.count, histogram.labels); histogram.buckets.forEach(interval -> { - writeMetric(buffer, metricFamily, "_bucket", interval.value, histogram.labels, interval.quantile.asHistogramLabel()); + if (!excludedHistoQuantiles.contains(interval.quantile)) { + writeMetric(buffer, metricFamily, "_bucket", interval.value, histogram.labels, interval.quantile.asHistogramLabel()); + } + }); writeMetric(buffer, metricFamily, "_bucket", histogram.count, histogram.labels, Interval.Quantile.POSITIVE_INFINITY.asHistogramLabel()); diff --git a/standalone/pom.xml b/standalone/pom.xml index d539c16..4bfaf8b 100644 --- a/standalone/pom.xml +++ b/standalone/pom.xml @@ -16,8 +16,7 @@ 18.0 3.4.0 - 4.0.47.Final - + 4.0.53.Final 1.2.3 1.7.16 diff --git a/standalone/src/main/java/com/zegelin/cassandra/exporter/Application.java b/standalone/src/main/java/com/zegelin/cassandra/exporter/Application.java index 6b57116..4d33f08 100644 --- a/standalone/src/main/java/com/zegelin/cassandra/exporter/Application.java +++ b/standalone/src/main/java/com/zegelin/cassandra/exporter/Application.java @@ -6,7 +6,6 @@ import com.datastax.driver.core.policies.RoundRobinPolicy; import com.datastax.driver.core.policies.WhiteListPolicy; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.zegelin.picocli.InetSocketAddressTypeConverter; import com.zegelin.picocli.JMXServiceURLTypeConverter; import com.zegelin.cassandra.exporter.cli.HarvesterOptions; @@ -78,6 +77,9 @@ protected int defaultPort() { @Option(names = "--cql-password", paramLabel = "PASSWORD", description = "CQL authentication password.") private String cqlPassword; + + @Option(names = "--cql-ssl", paramLabel = "SSL", description = "Creates enctrypted DB connections.") + private boolean ssl; @Option(names = {"-v", "--verbose"}, description = "Enable verbose logging. Multiple invocations increase the verbosity.") boolean[] verbosity = {}; @@ -145,6 +147,9 @@ private Cluster establishClusterConnection() { if (cqlUser != null && cqlPassword != null) { clusterBuilder.withCredentials(cqlUser, cqlPassword); } + if (ssl) { + clusterBuilder.withSSL(); + } final Cluster cluster = clusterBuilder.build(); diff --git a/standalone/src/main/java/com/zegelin/cassandra/exporter/JMXHarvester.java b/standalone/src/main/java/com/zegelin/cassandra/exporter/JMXHarvester.java index a5f8a40..674cd3e 100644 --- a/standalone/src/main/java/com/zegelin/cassandra/exporter/JMXHarvester.java +++ b/standalone/src/main/java/com/zegelin/cassandra/exporter/JMXHarvester.java @@ -1,6 +1,5 @@ package com.zegelin.cassandra.exporter; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import com.zegelin.cassandra.exporter.cli.HarvesterOptions; import com.zegelin.cassandra.exporter.collector.RemoteGossiperMBeanMetricFamilyCollector; @@ -12,6 +11,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; public class JMXHarvester extends Harvester { @@ -41,8 +41,7 @@ void reconcileMBeans() { // unregister { - final Set removedMBeans = Sets.difference(currentMBeans, newMBeans); - + final Set removedMBeans = currentMBeans.stream().filter(cmb -> !newMBeans.contains(cmb)).collect(Collectors.toSet()); logger.debug("Removing {} old MBeans.", removedMBeans.size()); for (final ObjectInstance instance : removedMBeans) { @@ -53,8 +52,7 @@ void reconcileMBeans() { // register { - final Set addedMBeans = Sets.difference(newMBeans, currentMBeans); - + final Set addedMBeans = newMBeans.stream().filter(nmb -> !currentMBeans.contains(nmb)).collect(Collectors.toSet()); logger.debug("Found {} new MBeans.", addedMBeans.size()); for (final ObjectInstance instance : addedMBeans) {