SOLR-9854: Collect metrics for index merges and index store IO. (squashed)

This commit is contained in:
Andrzej Bialecki 2017-01-03 11:11:50 +01:00
parent 7b2e3db553
commit 48ca9fc3f4
17 changed files with 945 additions and 48 deletions

View File

@ -212,6 +212,8 @@ New Features
* SOLR-9896: Instrument and collect metrics from query, update, core admin and core load thread pools. (shalin)
* SOLR-9854: Collect metrics for index merges and index store IO (ab)
Optimizations
----------------------
* SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

View File

@ -0,0 +1,511 @@
/*
* 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.core;
import java.io.IOException;
import java.util.Collection;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.LockFactory;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.util.plugin.SolrCoreAware;
/**
* An implementation of {@link DirectoryFactory} that decorates provided factory by
* adding metrics for directory IO operations.
*/
public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCoreAware {
private final SolrMetricManager metricManager;
private final String registry;
private final DirectoryFactory in;
private boolean directoryDetails = false;
public MetricsDirectoryFactory(SolrMetricManager metricManager, String registry, DirectoryFactory in) {
this.metricManager = metricManager;
this.registry = registry;
this.in = in;
}
public DirectoryFactory getDelegate() {
return in;
}
/**
* Currently the following arguments are supported:
* <ul>
* <li><code>directoryDetails</code> - (optional bool) when true then additional detailed metrics
* will be collected. These include eg. IO size histograms and per-file counters and histograms</li>
* </ul>
* @param args init args
*/
@Override
public void init(NamedList args) {
// should be already inited
// in.init(args);
if (args == null) {
return;
}
Boolean dd = args.getBooleanArg("directoryDetails");
if (dd != null) {
directoryDetails = dd;
} else {
directoryDetails = false;
}
}
@Override
public void doneWithDirectory(Directory dir) throws IOException {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.doneWithDirectory(dir);
}
@Override
public void addCloseListener(Directory dir, CachingDirectoryFactory.CloseListener closeListener) {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.addCloseListener(dir, closeListener);
}
@Override
public void close() throws IOException {
in.close();
}
@Override
protected Directory create(String path, LockFactory lockFactory, DirContext dirContext) throws IOException {
Directory dir = in.create(path, lockFactory, dirContext);
return new MetricsDirectory(metricManager, registry, dir, directoryDetails);
}
@Override
protected LockFactory createLockFactory(String rawLockType) throws IOException {
return in.createLockFactory(rawLockType);
}
@Override
public boolean exists(String path) throws IOException {
return in.exists(path);
}
@Override
public void remove(Directory dir) throws IOException {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.remove(dir);
}
@Override
public void remove(Directory dir, boolean afterCoreClose) throws IOException {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.remove(dir, afterCoreClose);
}
@Override
public boolean isSharedStorage() {
return in.isSharedStorage();
}
@Override
public boolean isAbsolute(String path) {
return in.isAbsolute(path);
}
@Override
public boolean searchersReserveCommitPoints() {
return in.searchersReserveCommitPoints();
}
@Override
public String getDataHome(CoreDescriptor cd) throws IOException {
return in.getDataHome(cd);
}
@Override
public long size(Directory directory) throws IOException {
return in.size(directory);
}
@Override
public long size(String path) throws IOException {
return in.size(path);
}
@Override
public Collection<SolrInfoMBean> offerMBeans() {
return in.offerMBeans();
}
@Override
public void cleanupOldIndexDirectories(String dataDirPath, String currentIndexDirPath) {
in.cleanupOldIndexDirectories(dataDirPath, currentIndexDirPath);
}
@Override
public void remove(String path, boolean afterCoreClose) throws IOException {
in.remove(path, afterCoreClose);
}
@Override
public void remove(String path) throws IOException {
in.remove(path);
}
@Override
public void move(Directory fromDir, Directory toDir, String fileName, IOContext ioContext) throws IOException {
in.move(fromDir, toDir, fileName, ioContext);
}
@Override
public Directory get(String path, DirContext dirContext, String rawLockType) throws IOException {
Directory dir = in.get(path, dirContext, rawLockType);
if (dir instanceof MetricsDirectory) {
return dir;
} else {
return new MetricsDirectory(metricManager, registry, dir, directoryDetails);
}
}
@Override
public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
super.renameWithOverwrite(dir, fileName, toName);
}
@Override
public String normalize(String path) throws IOException {
return in.normalize(path);
}
@Override
protected boolean deleteOldIndexDirectory(String oldDirPath) throws IOException {
return in.deleteOldIndexDirectory(oldDirPath);
}
@Override
public void initCoreContainer(CoreContainer cc) {
in.initCoreContainer(cc);
}
@Override
protected Directory getBaseDir(Directory dir) {
return in.getBaseDir(dir);
}
@Override
public void incRef(Directory dir) {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.incRef(dir);
}
@Override
public boolean isPersistent() {
return in.isPersistent();
}
@Override
public void inform(SolrCore core) {
if (in instanceof SolrCoreAware) {
((SolrCoreAware)in).inform(core);
}
}
@Override
public void release(Directory dir) throws IOException {
// unwrap
if (dir instanceof MetricsDirectory) {
dir = ((MetricsDirectory)dir).getDelegate();
}
in.release(dir);
}
private static final String SEGMENTS = "segments";
private static final String SEGMENTS_PREFIX = "segments_";
private static final String PENDING_SEGMENTS_PREFIX = "pending_segments_";
private static final String TEMP = "temp";
private static final String OTHER = "other";
public static class MetricsDirectory extends FilterDirectory {
private final Directory in;
private final String registry;
private final SolrMetricManager metricManager;
private final Meter totalReads;
private final Histogram totalReadSizes;
private final Meter totalWrites;
private final Histogram totalWriteSizes;
private final boolean directoryDetails;
private final String PREFIX = SolrInfoMBean.Category.DIRECTORY.toString() + ".";
public MetricsDirectory(SolrMetricManager metricManager, String registry, Directory in, boolean directoryDetails) throws IOException {
super(in);
this.metricManager = metricManager;
this.registry = registry;
this.in = in;
this.directoryDetails = directoryDetails;
this.totalReads = metricManager.meter(registry, "reads", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
this.totalWrites = metricManager.meter(registry, "writes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
if (directoryDetails) {
this.totalReadSizes = metricManager.histogram(registry, "readSizes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
this.totalWriteSizes = metricManager.histogram(registry, "writeSizes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
} else {
this.totalReadSizes = null;
this.totalWriteSizes = null;
}
}
private String getMetricName(String name, boolean output) {
if (!directoryDetails) {
return null;
}
String lastName;
if (name.startsWith(SEGMENTS_PREFIX) || name.startsWith(PENDING_SEGMENTS_PREFIX)) {
lastName = SEGMENTS;
} else {
int pos = name.lastIndexOf('.');
if (pos != -1 && name.length() > pos + 1) {
lastName = name.substring(pos + 1);
} else {
lastName = OTHER;
}
}
StringBuilder sb = new StringBuilder(PREFIX);
sb.append(lastName);
sb.append('.');
if (output) {
sb.append("write");
} else {
sb.append("read");
}
return sb.toString();
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
IndexOutput output = in.createOutput(name, context);
if (output != null) {
return new MetricsOutput(totalWrites, totalWriteSizes, metricManager, registry, getMetricName(name, true), output);
} else {
return null;
}
}
@Override
public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
IndexOutput output = in.createTempOutput(prefix, suffix, context);
if (output != null) {
return new MetricsOutput(totalWrites, totalWriteSizes, metricManager, registry, getMetricName(TEMP, true), output);
} else {
return null;
}
}
@Override
public IndexInput openInput(String name, IOContext context) throws IOException {
IndexInput input = in.openInput(name, context);
if (input != null) {
return new MetricsInput(totalReads, totalReadSizes, metricManager, registry, getMetricName(name, false), input);
} else {
return null;
}
}
}
public static class MetricsOutput extends IndexOutput {
private final IndexOutput in;
private final Histogram histogram;
private final Meter meter;
private final Meter totalMeter;
private final Histogram totalHistogram;
private final boolean withDetails;
public MetricsOutput(Meter totalMeter, Histogram totalHistogram, SolrMetricManager metricManager,
String registry, String metricName, IndexOutput in) {
super(in.toString(), in.getName());
this.in = in;
this.totalMeter = totalMeter;
this.totalHistogram = totalHistogram;
if (metricName != null && totalHistogram != null) {
withDetails = true;
String histName = metricName + "Sizes";
String meterName = metricName + "s";
this.histogram = metricManager.histogram(registry, histName);
this.meter = metricManager.meter(registry, meterName);
} else {
withDetails = false;
this.histogram = null;
this.meter = null;
}
}
@Override
public void writeByte(byte b) throws IOException {
in.writeByte(b);
totalMeter.mark();
if (withDetails) {
totalHistogram.update(1);
meter.mark();
histogram.update(1);
}
}
@Override
public void writeBytes(byte[] b, int offset, int length) throws IOException {
in.writeBytes(b, offset, length);
totalMeter.mark(length);
if (withDetails) {
totalHistogram.update(length);
meter.mark(length);
histogram.update(length);
}
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public long getFilePointer() {
return in.getFilePointer();
}
@Override
public long getChecksum() throws IOException {
return in.getChecksum();
}
}
public static class MetricsInput extends IndexInput {
private final IndexInput in;
private final Meter totalMeter;
private final Histogram totalHistogram;
private final Histogram histogram;
private final Meter meter;
private final boolean withDetails;
public MetricsInput(Meter totalMeter, Histogram totalHistogram, SolrMetricManager metricManager, String registry, String metricName, IndexInput in) {
super(in.toString());
this.in = in;
this.totalMeter = totalMeter;
this.totalHistogram = totalHistogram;
if (metricName != null && totalHistogram != null) {
withDetails = true;
String histName = metricName + "Sizes";
String meterName = metricName + "s";
this.histogram = metricManager.histogram(registry, histName);
this.meter = metricManager.meter(registry, meterName);
} else {
withDetails = false;
this.histogram = null;
this.meter = null;
}
}
public MetricsInput(Meter totalMeter, Histogram totalHistogram, Histogram histogram, Meter meter, IndexInput in) {
super(in.toString());
this.in = in;
this.totalMeter = totalMeter;
this.totalHistogram = totalHistogram;
this.histogram = histogram;
this.meter = meter;
if (totalHistogram != null && meter != null && histogram != null) {
withDetails = true;
} else {
withDetails = false;
}
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public long getFilePointer() {
return in.getFilePointer();
}
@Override
public void seek(long pos) throws IOException {
in.seek(pos);
}
@Override
public long length() {
return in.length();
}
@Override
public IndexInput clone() {
return new MetricsInput(totalMeter, totalHistogram, histogram, meter, in.clone());
}
@Override
public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
IndexInput slice = in.slice(sliceDescription, offset, length);
if (slice != null) {
return new MetricsInput(totalMeter, totalHistogram, histogram, meter, slice);
} else {
return null;
}
}
@Override
public byte readByte() throws IOException {
totalMeter.mark();
if (withDetails) {
totalHistogram.update(1);
meter.mark();
histogram.update(1);
}
return in.readByte();
}
@Override
public void readBytes(byte[] b, int offset, int len) throws IOException {
totalMeter.mark(len);
if (withDetails) {
totalHistogram.update(len);
meter.mark(len);
histogram.update(len);
}
in.readBytes(b, offset, len);
}
}
}

View File

@ -642,8 +642,15 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
dirFactory = new NRTCachingDirectoryFactory();
dirFactory.initCoreContainer(getCoreDescriptor().getCoreContainer());
}
if (solrConfig.indexConfig.metricsInfo != null && solrConfig.indexConfig.metricsInfo.isEnabled()) {
final DirectoryFactory factory = new MetricsDirectoryFactory(coreDescriptor.getCoreContainer().getMetricManager(),
coreMetricManager.getRegistryName(), dirFactory);
factory.init(solrConfig.indexConfig.metricsInfo.initArgs);
return factory;
} else {
return dirFactory;
}
}
private void initIndexReaderFactory() {
IndexReaderFactory indexReaderFactory;
@ -846,6 +853,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
resourceLoader = config.getResourceLoader();
this.solrConfig = config;
this.configSetProperties = configSetProperties;
// Initialize the metrics manager
this.coreMetricManager = initCoreMetricManager(config);
if (updateHandler == null) {
directoryFactory = initDirectoryFactory();
@ -863,17 +872,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
checkVersionFieldExistsInSchema(schema, coreDescriptor);
// Initialize the metrics manager
this.coreMetricManager = initCoreMetricManager(config);
SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
// initialize searcher-related metrics
newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcher");
newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherTime");
newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherWarmup");
newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherMaxReached");
newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherErrors");
newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "new", Category.SEARCHER.toString());
newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "time", Category.SEARCHER.toString(), "new");
newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "warmup", Category.SEARCHER.toString(), "new");
newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "maxReached", Category.SEARCHER.toString(), "new");
newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "errors", Category.SEARCHER.toString(), "new");
// Initialize JMX
this.infoRegistry = initInfoRegistry(name, config);

View File

@ -114,6 +114,9 @@ public class SolrDeletionPolicy extends IndexDeletionPolicy implements NamedList
protected void appendDetails(StringBuilder sb, IndexCommit c) {
Directory dir = c.getDirectory();
if (dir instanceof MetricsDirectoryFactory.MetricsDirectory) { // unwrap
dir = ((MetricsDirectoryFactory.MetricsDirectory) dir).getDelegate();
}
if (dir instanceof FSDirectory) {
FSDirectory fsd = (FSDirectory) dir;
sb.append("dir=").append(fsd.getDirectory());
@ -194,6 +197,9 @@ public class SolrDeletionPolicy extends IndexDeletionPolicy implements NamedList
private String getId(IndexCommit commit) {
StringBuilder sb = new StringBuilder();
Directory dir = commit.getDirectory();
if (dir instanceof MetricsDirectoryFactory.MetricsDirectory) { // unwrap
dir = ((MetricsDirectoryFactory.MetricsDirectory) dir).getDelegate();
}
// For anything persistent, make something that will
// be the same, regardless of the Directory instance.

View File

@ -32,7 +32,7 @@ public interface SolrInfoMBean {
/**
* Category of {@link SolrCore} component.
*/
enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER }
enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, OTHER }
/**
* Top-level group of beans for a subsystem.

View File

@ -18,6 +18,7 @@ package org.apache.solr.update;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -78,6 +79,7 @@ public class SolrIndexConfig implements MapSerializable {
public final PluginInfo mergePolicyInfo;
public final PluginInfo mergePolicyFactoryInfo;
public final PluginInfo mergeSchedulerInfo;
public final PluginInfo metricsInfo;
public final PluginInfo mergedSegmentWarmerInfo;
@ -99,6 +101,8 @@ public class SolrIndexConfig implements MapSerializable {
mergePolicyFactoryInfo = null;
mergeSchedulerInfo = null;
mergedSegmentWarmerInfo = null;
// enable coarse-grained metrics by default
metricsInfo = new PluginInfo("metrics", Collections.emptyMap(), null, null);
}
/**
@ -144,6 +148,12 @@ public class SolrIndexConfig implements MapSerializable {
writeLockTimeout=solrConfig.getInt(prefix+"/writeLockTimeout", def.writeLockTimeout);
lockType=solrConfig.get(prefix+"/lockType", def.lockType);
List<PluginInfo> infos = solrConfig.readPluginInfos(prefix + "/metrics", false, false);
if (infos.isEmpty()) {
metricsInfo = def.metricsInfo;
} else {
metricsInfo = infos.get(0);
}
mergeSchedulerInfo = getPluginInfo(prefix + "/mergeScheduler", solrConfig, def.mergeSchedulerInfo);
mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
mergePolicyFactoryInfo = getPluginInfo(prefix + "/mergePolicyFactory", solrConfig, def.mergePolicyFactoryInfo);
@ -197,6 +207,9 @@ public class SolrIndexConfig implements MapSerializable {
"lockType", lockType,
"infoStreamEnabled", infoStream != InfoStream.NO_OUTPUT);
if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo);
if (metricsInfo != null) {
m.put("metrics", metricsInfo);
}
if (mergePolicyInfo != null) {
m.put("mergePolicy", mergePolicyInfo);
} else if (mergePolicyFactoryInfo != null) {
@ -237,7 +250,8 @@ public class SolrIndexConfig implements MapSerializable {
iwc.setSimilarity(schema.getSimilarity());
MergePolicy mergePolicy = buildMergePolicy(schema);
iwc.setMergePolicy(mergePolicy);
iwc.setMergeScheduler(buildMergeScheduler(schema));
MergeScheduler mergeScheduler = buildMergeScheduler(schema);
iwc.setMergeScheduler(mergeScheduler);
iwc.setInfoStream(infoStream);
if (mergePolicy instanceof SortingMergePolicy) {

View File

@ -20,12 +20,19 @@ import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import com.codahale.metrics.Counter;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Meter;
import com.codahale.metrics.Timer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.IndexDeletionPolicy;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.InfoStream;
import org.apache.solr.common.util.IOUtils;
@ -33,6 +40,8 @@ import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.DirectoryFactory.DirContext;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.schema.IndexSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -60,6 +69,28 @@ public class SolrIndexWriter extends IndexWriter {
private InfoStream infoStream;
private Directory directory;
// metrics
private long majorMergeDocs = 512 * 1024;
private final Timer majorMerge;
private final Timer minorMerge;
private final Meter majorMergedDocs;
private final Meter majorDeletedDocs;
private final Counter mergeErrors;
private final Meter flushMeter; // original counter is package-private in IndexWriter
private final boolean mergeDetails;
private final AtomicInteger runningMajorMerges = new AtomicInteger();
private final Gauge<Integer> runningMajorMergesGauge;
private final AtomicInteger runningMinorMerges = new AtomicInteger();
private final Gauge<Integer> runningMinorMergesGauge;
private final AtomicInteger runningMajorMergesSegments = new AtomicInteger();
private final Gauge<Integer> runningMajorMergesSegmentsGauge;
private final AtomicInteger runningMinorMergesSegments = new AtomicInteger();
private final Gauge<Integer> runningMinorMergesSegmentsGauge;
private final AtomicLong runningMajorMergesDocs = new AtomicLong();
private final Gauge<Long> runningMajorMergesDocsGauge;
private final AtomicLong runningMinorMergesDocs = new AtomicLong();
private final Gauge<Long> runningMinorMergesDocsGauge;
public static SolrIndexWriter create(SolrCore core, String name, String path, DirectoryFactory directoryFactory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException {
SolrIndexWriter w = null;
@ -84,6 +115,20 @@ public class SolrIndexWriter extends IndexWriter {
this.directory = d;
numOpens.incrementAndGet();
log.debug("Opened Writer " + name);
// no metrics
minorMerge = null;
majorMerge = null;
mergeErrors = null;
majorMergedDocs = null;
majorDeletedDocs = null;
runningMinorMergesGauge = null;
runningMinorMergesDocsGauge = null;
runningMinorMergesSegmentsGauge = null;
runningMajorMergesGauge = null;
runningMajorMergesDocsGauge = null;
runningMajorMergesSegmentsGauge = null;
flushMeter = null;
mergeDetails = false;
}
private SolrIndexWriter(SolrCore core, String name, String path, Directory directory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException {
@ -97,6 +142,51 @@ public class SolrIndexWriter extends IndexWriter {
infoStream = getConfig().getInfoStream();
this.directory = directory;
numOpens.incrementAndGet();
SolrMetricManager metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
String registry = core.getCoreMetricManager().getRegistryName();
minorMerge = metricManager.timer(registry, "minor", SolrInfoMBean.Category.INDEX.toString(), "merge");
majorMerge = metricManager.timer(registry, "major", SolrInfoMBean.Category.INDEX.toString(), "merge");
mergeErrors = metricManager.counter(registry, "errors", SolrInfoMBean.Category.INDEX.toString(), "merge");
runningMajorMergesGauge = () -> runningMajorMerges.get();
runningMinorMergesGauge = () -> runningMinorMerges.get();
runningMajorMergesDocsGauge = () -> runningMajorMergesDocs.get();
runningMinorMergesDocsGauge = () -> runningMinorMergesDocs.get();
runningMajorMergesSegmentsGauge = () -> runningMajorMergesSegments.get();
runningMinorMergesSegmentsGauge = () -> runningMinorMergesSegments.get();
metricManager.register(registry, runningMajorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
metricManager.register(registry, runningMinorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
metricManager.register(registry, runningMajorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
metricManager.register(registry, runningMinorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
metricManager.register(registry, runningMajorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
metricManager.register(registry, runningMinorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
flushMeter = metricManager.meter(registry, "flush", SolrInfoMBean.Category.INDEX.toString());
if (config.metricsInfo != null && config.metricsInfo.initArgs != null) {
Object v = config.metricsInfo.initArgs.get("majorMergeDocs");
if (v != null) {
try {
majorMergeDocs = Long.parseLong(String.valueOf(v));
} catch (Exception e) {
log.warn("Invalid 'majorMergeDocs' argument, using default 512k", e);
}
}
Boolean Details = config.metricsInfo.initArgs.getBooleanArg("mergeDetails");
if (Details != null) {
mergeDetails = Details;
} else {
mergeDetails = false;
}
if (mergeDetails) {
majorMergedDocs = metricManager.meter(registry, "docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
majorDeletedDocs = metricManager.meter(registry, "deletedDocs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
} else {
majorMergedDocs = null;
majorDeletedDocs = null;
}
} else {
mergeDetails = false;
majorMergedDocs = null;
majorDeletedDocs = null;
}
}
@SuppressForbidden(reason = "Need currentTimeMillis, commit time should be used only for debugging purposes, " +
@ -112,6 +202,60 @@ public class SolrIndexWriter extends IndexWriter {
this.directoryFactory = factory;
}
// we override this method to collect metrics for merges.
@Override
public void merge(MergePolicy.OneMerge merge) throws IOException {
long deletedDocs = 0;
long totalNumDocs = merge.totalNumDocs();
for (SegmentCommitInfo info : merge.segments) {
totalNumDocs -= info.getDelCount();
deletedDocs += info.getDelCount();
}
boolean major = totalNumDocs > majorMergeDocs;
int segmentsCount = merge.segments.size();
Timer.Context context;
if (major) {
runningMajorMerges.incrementAndGet();
runningMajorMergesDocs.addAndGet(totalNumDocs);
runningMajorMergesSegments.addAndGet(segmentsCount);
if (mergeDetails) {
majorMergedDocs.mark(totalNumDocs);
majorDeletedDocs.mark(deletedDocs);
}
context = majorMerge.time();
} else {
runningMinorMerges.incrementAndGet();
runningMinorMergesDocs.addAndGet(totalNumDocs);
runningMinorMergesSegments.addAndGet(segmentsCount);
context = minorMerge.time();
}
try {
super.merge(merge);
} catch (Throwable t) {
mergeErrors.inc();
throw t;
} finally {
context.stop();
if (major) {
runningMajorMerges.decrementAndGet();
runningMajorMergesDocs.addAndGet(-totalNumDocs);
runningMajorMergesSegments.addAndGet(-segmentsCount);
} else {
runningMinorMerges.decrementAndGet();
runningMinorMergesDocs.addAndGet(-totalNumDocs);
runningMinorMergesSegments.addAndGet(-segmentsCount);
}
}
}
@Override
protected void doAfterFlush() throws IOException {
if (flushMeter != null) { // this is null when writer is used only for snapshot cleanup
flushMeter.mark();
}
super.doAfterFlush();
}
/**
* use DocumentBuilder now...
* private final void addField(Document doc, String name, String val) {

View File

@ -40,7 +40,7 @@ import org.apache.solr.common.util.NamedList;
public class MetricUtils {
/**
* Adds metrics from a Timer to a NamedList, using well-known names.
* Adds metrics from a Timer to a NamedList, using well-known back-compat names.
* @param lst The NamedList to add the metrics data to
* @param timer The Timer to extract the metrics from
*/
@ -68,7 +68,7 @@ public class MetricUtils {
}
/**
* Returns a NamedList respresentation of the given metric registry. Only those metrics
* Returns a NamedList representation of the given metric registry. Only those metrics
* are converted to NamedList which match at least one of the given MetricFilter instances.
*
* @param registry the {@link MetricRegistry} to be converted to NamedList
@ -104,31 +104,74 @@ public class MetricUtils {
static NamedList histogramToNamedList(Histogram histogram) {
NamedList response = new NamedList();
Snapshot snapshot = histogram.getSnapshot();
response.add("requests", histogram.getCount());
response.add("minTime", nsToMs(snapshot.getMin()));
response.add("maxTime", nsToMs(snapshot.getMax()));
response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
response.add("count", histogram.getCount());
// non-time based values
addSnapshot(response, snapshot, false);
return response;
}
// optionally convert ns to ms
static double nsToMs(boolean convert, double value) {
if (convert) {
return nsToMs(value);
} else {
return value;
}
}
static final String MS = "_ms";
static final String MIN = "min";
static final String MIN_MS = MIN + MS;
static final String MAX = "max";
static final String MAX_MS = MAX + MS;
static final String MEAN = "mean";
static final String MEAN_MS = MEAN + MS;
static final String MEDIAN = "median";
static final String MEDIAN_MS = MEDIAN + MS;
static final String STDDEV = "stddev";
static final String STDDEV_MS = STDDEV + MS;
static final String P75 = "p75";
static final String P75_MS = P75 + MS;
static final String P95 = "p95";
static final String P95_MS = P95 + MS;
static final String P99 = "p99";
static final String P99_MS = P99 + MS;
static final String P999 = "p999";
static final String P999_MS = P999 + MS;
// some snapshots represent time in ns, other snapshots represent raw values (eg. chunk size)
static void addSnapshot(NamedList response, Snapshot snapshot, boolean ms) {
response.add((ms ? MIN_MS: MIN), nsToMs(ms, snapshot.getMin()));
response.add((ms ? MAX_MS: MAX), nsToMs(ms, snapshot.getMax()));
response.add((ms ? MEAN_MS : MEAN), nsToMs(ms, snapshot.getMean()));
response.add((ms ? MEDIAN_MS: MEDIAN), nsToMs(ms, snapshot.getMedian()));
response.add((ms ? STDDEV_MS: STDDEV), nsToMs(ms, snapshot.getStdDev()));
response.add((ms ? P75_MS: P75), nsToMs(ms, snapshot.get75thPercentile()));
response.add((ms ? P95_MS: P95), nsToMs(ms, snapshot.get95thPercentile()));
response.add((ms ? P99_MS: P99), nsToMs(ms, snapshot.get99thPercentile()));
response.add((ms ? P999_MS: P999), nsToMs(ms, snapshot.get999thPercentile()));
}
static NamedList timerToNamedList(Timer timer) {
NamedList response = new NamedList();
addMetrics(response, timer);
response.add("count", timer.getCount());
response.add("meanRate", timer.getMeanRate());
response.add("1minRate", timer.getOneMinuteRate());
response.add("5minRate", timer.getFiveMinuteRate());
response.add("15minRate", timer.getFifteenMinuteRate());
// time-based values in nanoseconds
addSnapshot(response, timer.getSnapshot(), true);
return response;
}
static NamedList meterToNamedList(Meter meter) {
NamedList response = new NamedList();
response.add("requests", meter.getCount());
response.add("avgRequestsPerSecond", meter.getMeanRate());
response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
response.add("count", meter.getCount());
response.add("meanRate", meter.getMeanRate());
response.add("1minRate", meter.getOneMinuteRate());
response.add("5minRate", meter.getFiveMinuteRate());
response.add("15minRate", meter.getFifteenMinuteRate());
return response;
}
@ -140,7 +183,7 @@ public class MetricUtils {
static NamedList counterToNamedList(Counter counter) {
NamedList response = new NamedList();
response.add("requests", counter.getCount());
response.add("count", counter.getCount());
return response;
}

View File

@ -0,0 +1,57 @@
<?xml version="1.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.
-->
<config>
<dataDir>${solr.data.dir:}</dataDir>
<directoryFactory name="DirectoryFactory"
class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
<schemaFactory class="ClassicIndexSchemaFactory"/>
<luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
<indexConfig>
<metrics>
<bool name="directoryDetails">${solr.tests.directoryDetails:false}</bool>
<bool name="mergeDetails">${solr.tests.mergeDetails:false}</bool>
</metrics>
<!-- intentionally set very low values here to trigger multiple flushes and merges.
DO NOT USE THESE ABSURD VALUES IN PRODUCTION. -->
<mergeFactor>3</mergeFactor>
<maxBufferedDocs>100</maxBufferedDocs>
<mergePolicy class="org.apache.lucene.index.TieredMergePolicy"/>
</indexConfig>
<updateHandler class="solr.DirectUpdateHandler2">
<commitWithin>
<softCommit>${solr.commitwithin.softcommit:true}</softCommit>
</commitWithin>
</updateHandler>
<requestHandler name="/select" class="solr.SearchHandler">
<lst name="defaults">
<str name="echoParams">explicit</str>
<str name="indent">true</str>
<str name="df">text</str>
</lst>
</requestHandler>
</config>

View File

@ -128,6 +128,8 @@ public class TestConfig extends SolrTestCaseJ4 {
++numDefaultsTested; assertEquals("default infoStream", InfoStream.NO_OUTPUT, sic.infoStream);
++numDefaultsTested; assertNotNull("default metrics", sic.metricsInfo);
// mergePolicyInfo and mergePolicyFactoryInfo are mutually exclusive
// so ++ count them only once for both instead of individually
++numDefaultsTested; ++numNullDefaults;

View File

@ -66,6 +66,8 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.CachingDirectoryFactory;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.MetricsDirectoryFactory;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.StandardDirectoryFactory;
import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
@ -895,12 +897,21 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
}
}
private CachingDirectoryFactory getCachingDirectoryFactory(SolrCore core) {
DirectoryFactory df = core.getDirectoryFactory();
if (df instanceof MetricsDirectoryFactory) {
return (CachingDirectoryFactory)((MetricsDirectoryFactory)df).getDelegate();
} else {
return (CachingDirectoryFactory)df;
}
}
private void checkForSingleIndex(JettySolrRunner jetty) {
CoreContainer cores = jetty.getCoreContainer();
Collection<SolrCore> theCores = cores.getCores();
for (SolrCore core : theCores) {
String ddir = core.getDataDir();
CachingDirectoryFactory dirFactory = (CachingDirectoryFactory) core.getDirectoryFactory();
CachingDirectoryFactory dirFactory = getCachingDirectoryFactory(core);
synchronized (dirFactory) {
Set<String> livePaths = dirFactory.getLivePaths();
// one for data, one for hte index under data and one for the snapshot metadata.

View File

@ -24,6 +24,8 @@ import org.apache.lucene.store.LockFactory;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.MetricsDirectoryFactory;
import org.apache.solr.core.MockFSDirectoryFactory;
import org.apache.solr.core.SolrCore;
import org.apache.solr.response.SolrQueryResponse;
@ -75,7 +77,13 @@ public class CoreMergeIndexesAdminHandlerTest extends SolrTestCaseJ4 {
final CoreAdminHandler admin = new CoreAdminHandler(cores);
try (SolrCore core = cores.getCore("collection1")) {
FailingDirectoryFactory dirFactory = (FailingDirectoryFactory)core.getDirectoryFactory();
DirectoryFactory df = core.getDirectoryFactory();
FailingDirectoryFactory dirFactory;
if (df instanceof MetricsDirectoryFactory) {
dirFactory = (FailingDirectoryFactory)((MetricsDirectoryFactory)df).getDelegate();
} else {
dirFactory = (FailingDirectoryFactory)df;
}
try {
dirFactory.fail = true;

View File

@ -49,9 +49,9 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
assertNotNull(values.get("solr.node"));
NamedList nl = (NamedList) values.get("solr.core.collection1");
assertNotNull(nl);
assertNotNull(nl.get("newSearcherErrors")); // counter type
assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
assertNotNull(nl.get("SEARCHER.new.errors")); // counter type
assertNotNull(((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
assertEquals(0L, ((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
nl = (NamedList) values.get("solr.node");
assertNotNull(nl.get("cores.loaded")); // int gauge
assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));

View File

@ -197,6 +197,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
} else {
assertNull(m.get("mergedSegmentWarmer"));
}
++mSizeExpected; assertNotNull(m.get("metrics"));
assertEquals(mSizeExpected, m.size());
}

View File

@ -0,0 +1,94 @@
/*
* 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.update;
import java.util.Map;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import com.codahale.metrics.Metric;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.request.SolrQueryRequest;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test proper registration and collection of index and directory metrics.
*/
public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
@BeforeClass
public static void beforeClass() throws Exception {
System.setProperty("solr.tests.mergeDetails", "true");
System.setProperty("solr.tests.directoryDetails", "true");
initCore("solrconfig-indexmetrics.xml", "schema.xml");
}
@Test
public void testIndexMetrics() throws Exception {
SolrQueryRequest req = lrf.makeRequest();
UpdateHandler uh = req.getCore().getUpdateHandler();
AddUpdateCommand add = new AddUpdateCommand(req);
for (int i = 0; i < 1000; i++) {
add.clear();
add.solrDoc = new SolrInputDocument();
add.solrDoc.addField("id", "" + i);
add.solrDoc.addField("foo_s", "foo-" + i);
uh.addDoc(add);
}
uh.commit(new CommitUpdateCommand(req, false));
MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName());
assertNotNull(registry);
// make sure all merges are finished
h.reload();
Map<String, Metric> metrics = registry.getMetrics();
assertTrue(metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count() >= 12);
// this is variable, depending on the codec and the number of created files
assertTrue(metrics.entrySet().stream().filter(e -> e.getKey().startsWith("DIRECTORY")).count() > 50);
// check basic index meters
Timer timer = (Timer)metrics.get("INDEX.merge.minor");
assertEquals("minorMerge: " + timer.getCount(), 4, timer.getCount());
timer = (Timer)metrics.get("INDEX.merge.major");
assertEquals("majorMerge: " + timer.getCount(), 0, timer.getCount());
Meter meter = (Meter)metrics.get("INDEX.merge.major.docs");
assertEquals("majorMergeDocs: " + meter.getCount(), 0, meter.getCount());
meter = (Meter)metrics.get("INDEX.flush");
assertEquals("flush: " + meter.getCount(), 19, meter.getCount());
// check basic directory meters
meter = (Meter)metrics.get("DIRECTORY.total.reads");
assertTrue("totalReads", meter.getCount() > 0);
meter = (Meter)metrics.get("DIRECTORY.total.writes");
assertTrue("totalWrites", meter.getCount() > 0);
Histogram histogram = (Histogram)metrics.get("DIRECTORY.total.readSizes");
assertTrue("readSizes", histogram.getCount() > 0);
histogram = (Histogram)metrics.get("DIRECTORY.total.writeSizes");
assertTrue("writeSizes", histogram.getCount() > 0);
// check detailed meters
meter = (Meter)metrics.get("DIRECTORY.segments.writes");
assertTrue("segmentsWrites", meter.getCount() > 0);
histogram = (Histogram)metrics.get("DIRECTORY.segments.writeSizes");
assertTrue("segmentsWriteSizes", histogram.getCount() > 0);
}
}

View File

@ -23,7 +23,6 @@ import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.junit.Test;
public class MetricUtilsTest extends SolrTestCaseJ4 {
@ -34,24 +33,23 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
final Timer timer = new Timer();
final int iterations = random().nextInt(100);
for (int i = 0; i < iterations; ++i) {
timer.update(random().nextInt(), TimeUnit.NANOSECONDS);
timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS);
}
// obtain timer metrics
final NamedList<Object> lst = new SimpleOrderedMap<>();
MetricUtils.addMetrics(lst, timer);
NamedList lst = MetricUtils.timerToNamedList(timer);
// check that expected metrics were obtained
assertEquals(lst.size(), 9);
assertEquals(14, lst.size());
final Snapshot snapshot = timer.getSnapshot();
// cannot test avgRequestsPerMinute directly because mean rate changes as time increases!
// assertEquals(lst.get("avgRequestsPerSecond"), timer.getMeanRate());
assertEquals(lst.get("5minRateRequestsPerSecond"), timer.getFiveMinuteRate());
assertEquals(lst.get("15minRateRequestsPerSecond"), timer.getFifteenMinuteRate());
assertEquals(lst.get("avgTimePerRequest"), MetricUtils.nsToMs(snapshot.getMean()));
assertEquals(lst.get("medianRequestTime"), MetricUtils.nsToMs(snapshot.getMedian()));
assertEquals(lst.get("75thPcRequestTime"), MetricUtils.nsToMs(snapshot.get75thPercentile()));
assertEquals(lst.get("95thPcRequestTime"), MetricUtils.nsToMs(snapshot.get95thPercentile()));
assertEquals(lst.get("99thPcRequestTime"), MetricUtils.nsToMs(snapshot.get99thPercentile()));
assertEquals(lst.get("999thPcRequestTime"), MetricUtils.nsToMs(snapshot.get999thPercentile()));
assertEquals(timer.getFiveMinuteRate(), lst.get("5minRate"));
assertEquals(timer.getFifteenMinuteRate(), lst.get("15minRate"));
assertEquals(MetricUtils.nsToMs(snapshot.getMean()), lst.get("mean_ms"));
assertEquals(MetricUtils.nsToMs(snapshot.getMedian()), lst.get("median_ms"));
assertEquals(MetricUtils.nsToMs(snapshot.get75thPercentile()), lst.get("p75_ms"));
assertEquals(MetricUtils.nsToMs(snapshot.get95thPercentile()), lst.get("p95_ms"));
assertEquals(MetricUtils.nsToMs(snapshot.get99thPercentile()), lst.get("p99_ms"));
assertEquals(MetricUtils.nsToMs(snapshot.get999thPercentile()), lst.get("p999_ms"));
}
}