mirror of https://github.com/apache/lucene.git
SOLR prometheus: simplify concurrent collection (#1723)
No semantic difference in behavior.
This commit is contained in:
parent
ec1c5cfffe
commit
e6a11f8c3a
|
@ -18,7 +18,7 @@
|
||||||
package org.apache.solr.prometheus.collector;
|
package org.apache.solr.prometheus.collector;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
@ -29,12 +29,12 @@ import org.apache.solr.prometheus.scraper.SolrScraper;
|
||||||
public class MetricsCollectorFactory {
|
public class MetricsCollectorFactory {
|
||||||
|
|
||||||
private final MetricsConfiguration metricsConfiguration;
|
private final MetricsConfiguration metricsConfiguration;
|
||||||
private final Executor executor;
|
private final ExecutorService executor;
|
||||||
private final int refreshInSeconds;
|
private final int refreshInSeconds;
|
||||||
private final SolrScraper solrScraper;
|
private final SolrScraper solrScraper;
|
||||||
|
|
||||||
public MetricsCollectorFactory(
|
public MetricsCollectorFactory(
|
||||||
Executor executor,
|
ExecutorService executor,
|
||||||
int refreshInSeconds,
|
int refreshInSeconds,
|
||||||
SolrScraper solrScraper,
|
SolrScraper solrScraper,
|
||||||
MetricsConfiguration metricsConfiguration) {
|
MetricsConfiguration metricsConfiguration) {
|
||||||
|
|
|
@ -19,20 +19,20 @@ package org.apache.solr.prometheus.collector;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import io.prometheus.client.Collector;
|
import io.prometheus.client.Collector;
|
||||||
import io.prometheus.client.Histogram;
|
import io.prometheus.client.Histogram;
|
||||||
import org.apache.solr.prometheus.exporter.SolrExporter;
|
import org.apache.solr.prometheus.exporter.SolrExporter;
|
||||||
import org.apache.solr.prometheus.scraper.Async;
|
|
||||||
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -53,7 +53,7 @@ public class SchedulerMetricsCollector implements Closeable {
|
||||||
1,
|
1,
|
||||||
new SolrNamedThreadFactory("scheduled-metrics-collector"));
|
new SolrNamedThreadFactory("scheduled-metrics-collector"));
|
||||||
|
|
||||||
private final Executor executor;
|
private final ExecutorService executor;
|
||||||
|
|
||||||
private final List<Observer> observers = new CopyOnWriteArrayList<>();
|
private final List<Observer> observers = new CopyOnWriteArrayList<>();
|
||||||
|
|
||||||
|
@ -63,7 +63,7 @@ public class SchedulerMetricsCollector implements Closeable {
|
||||||
.register(SolrExporter.defaultRegistry);
|
.register(SolrExporter.defaultRegistry);
|
||||||
|
|
||||||
public SchedulerMetricsCollector(
|
public SchedulerMetricsCollector(
|
||||||
Executor executor,
|
ExecutorService executor,
|
||||||
int duration,
|
int duration,
|
||||||
TimeUnit timeUnit,
|
TimeUnit timeUnit,
|
||||||
List<MetricCollector> metricCollectors) {
|
List<MetricCollector> metricCollectors) {
|
||||||
|
@ -83,31 +83,27 @@ public class SchedulerMetricsCollector implements Closeable {
|
||||||
try (Histogram.Timer timer = metricsCollectionTime.startTimer()) {
|
try (Histogram.Timer timer = metricsCollectionTime.startTimer()) {
|
||||||
log.info("Beginning metrics collection");
|
log.info("Beginning metrics collection");
|
||||||
|
|
||||||
List<CompletableFuture<MetricSamples>> futures = new ArrayList<>();
|
final List<Future<MetricSamples>> futures = executor.invokeAll(
|
||||||
|
metricCollectors.stream()
|
||||||
for (MetricCollector metricsCollector : metricCollectors) {
|
.map(metricCollector -> (Callable<MetricSamples>) metricCollector::collect)
|
||||||
futures.add(CompletableFuture.supplyAsync(() -> {
|
.collect(Collectors.toList())
|
||||||
try {
|
);
|
||||||
return metricsCollector.collect();
|
MetricSamples metricSamples = new MetricSamples();
|
||||||
} catch (Exception e) {
|
for (Future<MetricSamples> future : futures) {
|
||||||
throw new RuntimeException(e);
|
try {
|
||||||
}
|
metricSamples.addAll(future.get());
|
||||||
}, executor));
|
} catch (ExecutionException e) {
|
||||||
|
log.error("Error occurred during metrics collection", e.getCause());//logok
|
||||||
|
// continue any ways; do not fail
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
notifyObservers(metricSamples.asList());
|
||||||
CompletableFuture<List<MetricSamples>> sampleFuture = Async.waitForAllSuccessfulResponses(futures);
|
|
||||||
List<MetricSamples> samples = sampleFuture.get();
|
|
||||||
|
|
||||||
MetricSamples metricSamples = new MetricSamples();
|
log.info("Completed metrics collection");
|
||||||
samples.forEach(metricSamples::addAll);
|
} catch (InterruptedException e) {
|
||||||
|
log.warn("Interrupted waiting for metric collection to complete", e);
|
||||||
notifyObservers(metricSamples.asList());
|
Thread.currentThread().interrupt();
|
||||||
|
|
||||||
log.info("Completed metrics collection");
|
|
||||||
} catch (InterruptedException | ExecutionException e) {
|
|
||||||
log.error("Error while waiting for metric collection to complete", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,61 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You 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 org.apache.solr.prometheus.scraper;
|
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.CompletableFuture;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
public class Async {
|
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
|
||||||
|
|
||||||
@SuppressWarnings({"rawtypes"})
|
|
||||||
public static <T> CompletableFuture<List<T>> waitForAllSuccessfulResponses(List<CompletableFuture<T>> futures) {
|
|
||||||
CompletableFuture<Void> completed = CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
|
|
||||||
|
|
||||||
return completed.thenApply(values -> {
|
|
||||||
return futures.stream()
|
|
||||||
.map(CompletableFuture::join)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
).exceptionally(error -> {
|
|
||||||
futures.stream()
|
|
||||||
.filter(CompletableFuture::isCompletedExceptionally)
|
|
||||||
.forEach(future -> {
|
|
||||||
try {
|
|
||||||
future.get();
|
|
||||||
} catch (Exception exception) {
|
|
||||||
log.warn("Error occurred during metrics collection", exception);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
return futures.stream()
|
|
||||||
.filter(future -> !(future.isCompletedExceptionally() || future.isCancelled()))
|
|
||||||
.map(CompletableFuture::join)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
|
@ -21,7 +21,7 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -44,7 +44,7 @@ public class SolrCloudScraper extends SolrScraper {
|
||||||
|
|
||||||
private Cache<String, HttpSolrClient> hostClientCache = CacheBuilder.newBuilder().build();
|
private Cache<String, HttpSolrClient> hostClientCache = CacheBuilder.newBuilder().build();
|
||||||
|
|
||||||
public SolrCloudScraper(CloudSolrClient solrClient, Executor executor, SolrClientFactory solrClientFactory) {
|
public SolrCloudScraper(CloudSolrClient solrClient, ExecutorService executor, SolrClientFactory solrClientFactory) {
|
||||||
super(executor);
|
super(executor);
|
||||||
this.solrClient = solrClient;
|
this.solrClient = solrClient;
|
||||||
this.solrClientFactory = solrClientFactory;
|
this.solrClientFactory = solrClientFactory;
|
||||||
|
|
|
@ -21,12 +21,11 @@ import java.io.IOException;
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executor;
|
|
||||||
import java.util.concurrent.Future;
|
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -42,7 +41,6 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
||||||
import org.apache.solr.client.solrj.request.QueryRequest;
|
import org.apache.solr.client.solrj.request.QueryRequest;
|
||||||
import org.apache.solr.common.util.NamedList;
|
import org.apache.solr.common.util.NamedList;
|
||||||
import org.apache.solr.common.util.Pair;
|
|
||||||
import org.apache.solr.prometheus.collector.MetricSamples;
|
import org.apache.solr.prometheus.collector.MetricSamples;
|
||||||
import org.apache.solr.prometheus.exporter.MetricsQuery;
|
import org.apache.solr.prometheus.exporter.MetricsQuery;
|
||||||
import org.apache.solr.prometheus.exporter.SolrExporter;
|
import org.apache.solr.prometheus.exporter.SolrExporter;
|
||||||
|
@ -59,7 +57,7 @@ public abstract class SolrScraper implements Closeable {
|
||||||
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
protected final Executor executor;
|
protected final ExecutorService executor;
|
||||||
|
|
||||||
public abstract Map<String, MetricSamples> metricsForAllHosts(MetricsQuery query) throws IOException;
|
public abstract Map<String, MetricSamples> metricsForAllHosts(MetricsQuery query) throws IOException;
|
||||||
|
|
||||||
|
@ -69,7 +67,7 @@ public abstract class SolrScraper implements Closeable {
|
||||||
public abstract MetricSamples search(MetricsQuery query) throws IOException;
|
public abstract MetricSamples search(MetricsQuery query) throws IOException;
|
||||||
public abstract MetricSamples collections(MetricsQuery metricsQuery) throws IOException;
|
public abstract MetricSamples collections(MetricsQuery metricsQuery) throws IOException;
|
||||||
|
|
||||||
public SolrScraper(Executor executor) {
|
public SolrScraper(ExecutorService executor) {
|
||||||
this.executor = executor;
|
this.executor = executor;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -77,17 +75,32 @@ public abstract class SolrScraper implements Closeable {
|
||||||
Collection<String> items,
|
Collection<String> items,
|
||||||
Function<String, MetricSamples> samplesCallable) throws IOException {
|
Function<String, MetricSamples> samplesCallable) throws IOException {
|
||||||
|
|
||||||
List<CompletableFuture<Pair<String, MetricSamples>>> futures = items.stream()
|
Map<String, MetricSamples> result = new HashMap<>(); // sync on this when adding to it below
|
||||||
.map(item -> CompletableFuture.supplyAsync(() -> new Pair<>(item, samplesCallable.apply(item)), executor))
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
|
|
||||||
Future<List<Pair<String, MetricSamples>>> allComplete = Async.waitForAllSuccessfulResponses(futures);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
return allComplete.get().stream().collect(Collectors.toMap(Pair::first, Pair::second));
|
// invoke each samplesCallable with each item and putting the results in the above "result" map.
|
||||||
} catch (InterruptedException | ExecutionException e) {
|
executor.invokeAll(
|
||||||
throw new IOException(e);
|
items.stream()
|
||||||
|
.map(item -> (Callable<MetricSamples>) () -> {
|
||||||
|
try {
|
||||||
|
final MetricSamples samples = samplesCallable.apply(item);
|
||||||
|
synchronized (result) {
|
||||||
|
result.put(item, samples);
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
// do NOT totally fail; just log and move on
|
||||||
|
log.warn("Error occurred during metrics collection", e);
|
||||||
|
}
|
||||||
|
return null;//not used
|
||||||
|
})
|
||||||
|
.collect(Collectors.toList())
|
||||||
|
);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected MetricSamples request(SolrClient client, MetricsQuery query) throws IOException {
|
protected MetricSamples request(SolrClient client, MetricsQuery query) throws IOException {
|
||||||
|
|
|
@ -22,7 +22,7 @@ import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.JsonNode;
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
import org.apache.solr.client.solrj.SolrServerException;
|
import org.apache.solr.client.solrj.SolrServerException;
|
||||||
|
@ -38,7 +38,7 @@ public class SolrStandaloneScraper extends SolrScraper {
|
||||||
|
|
||||||
private final HttpSolrClient solrClient;
|
private final HttpSolrClient solrClient;
|
||||||
|
|
||||||
public SolrStandaloneScraper(HttpSolrClient solrClient, Executor executor) {
|
public SolrStandaloneScraper(HttpSolrClient solrClient, ExecutorService executor) {
|
||||||
super(executor);
|
super(executor);
|
||||||
this.solrClient = solrClient;
|
this.solrClient = solrClient;
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,78 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You 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 org.apache.solr.prometheus.scraper;
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.CompletableFuture;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.junit.Test;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
public class AsyncTest {
|
|
||||||
|
|
||||||
private CompletableFuture<Integer> failedFuture() {
|
|
||||||
CompletableFuture<Integer> result = new CompletableFuture<>();
|
|
||||||
result.completeExceptionally(new RuntimeException("Some error"));
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void getAllResults() throws Exception {
|
|
||||||
List<Integer> expectedValues = Arrays.asList(1, 2, 3, 4, 5, 6, 7);
|
|
||||||
|
|
||||||
CompletableFuture<List<Integer>> results = Async.waitForAllSuccessfulResponses(
|
|
||||||
expectedValues.stream()
|
|
||||||
.map(CompletableFuture::completedFuture)
|
|
||||||
.collect(Collectors.toList()));
|
|
||||||
|
|
||||||
List<Integer> actualValues = results.get();
|
|
||||||
|
|
||||||
Collections.sort(expectedValues);
|
|
||||||
Collections.sort(actualValues);
|
|
||||||
|
|
||||||
assertEquals(expectedValues, actualValues);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void ignoresFailures() throws Exception {
|
|
||||||
CompletableFuture<List<Integer>> results = Async.waitForAllSuccessfulResponses(Arrays.asList(
|
|
||||||
CompletableFuture.completedFuture(1),
|
|
||||||
failedFuture()
|
|
||||||
));
|
|
||||||
|
|
||||||
List<Integer> values = results.get();
|
|
||||||
|
|
||||||
assertEquals(Collections.singletonList(1), values);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void allFuturesFail() throws Exception {
|
|
||||||
CompletableFuture<List<Integer>> results = Async.waitForAllSuccessfulResponses(Collections.singletonList(
|
|
||||||
failedFuture()
|
|
||||||
));
|
|
||||||
|
|
||||||
List<Integer> values = results.get();
|
|
||||||
|
|
||||||
assertTrue(values.isEmpty());
|
|
||||||
}
|
|
||||||
}
|
|
Loading…
Reference in New Issue