From commits-return-100967-archive-asf-public=cust-asf.ponee.io@lucene.apache.org Tue May 15 22:03:28 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 88034180634 for ; Tue, 15 May 2018 22:03:26 +0200 (CEST) Received: (qmail 18134 invoked by uid 500); 15 May 2018 20:03:25 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 18125 invoked by uid 99); 15 May 2018 20:03:25 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 15 May 2018 20:03:25 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1F718F6D3C; Tue, 15 May 2018 20:03:25 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ab@apache.org To: commits@lucene.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: lucene-solr:jira/solr-11779: SOLR-11779: Final patch. Date: Tue, 15 May 2018 20:03:25 +0000 (UTC) Repository: lucene-solr Updated Branches: refs/heads/jira/solr-11779 f9849c572 -> 2841f4420 SOLR-11779: Final patch. Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2841f442 Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2841f442 Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2841f442 Branch: refs/heads/jira/solr-11779 Commit: 2841f4420a69f6f22113cbf3be14b6e7899a1925 Parents: f9849c5 Author: Andrzej Bialecki Authored: Tue May 15 22:03:04 2018 +0200 Committer: Andrzej Bialecki Committed: Tue May 15 22:03:04 2018 +0200 ---------------------------------------------------------------------- .../solr/handler/admin/MetricsHandler.java | 1 - .../handler/admin/MetricsHistoryHandler.java | 74 +++++-- .../solr/metrics/rrd/SolrRrdBackendFactory.java | 58 +++++- .../apache/solr/metrics/rrd/package-info.java | 22 ++ .../cloud/MetricsHistoryIntegrationTest.java | 124 ++++++++++-- .../cloud/autoscaling/sim/SimCloudManager.java | 1 + .../sim/SimClusterStateProvider.java | 7 + .../admin/MetricsHistoryHandlerTest.java | 5 +- .../metrics/rrd/SolrRrdBackendFactoryTest.java | 3 + .../solr/util/MockSearchableSolrClient.java | 1 - solr/licenses/rrd4j-3.2.jar.sha1 | 1 + solr/licenses/rrd4j-LICENSE-ASL.txt | 202 +++++++++++++++++++ solr/licenses/rrd4j-NOTICE.txt | 2 + 13 files changed, 451 insertions(+), 50 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java ---------------------------------------------------------------------- diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java index 6f1c5e0..ca291e8 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java @@ -41,7 +41,6 @@ import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.common.util.StrUtils; -import org.apache.solr.core.CoreContainer; import org.apache.solr.handler.RequestHandlerBase; import org.apache.solr.metrics.SolrMetricManager; import org.apache.solr.request.SolrQueryRequest; http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java ---------------------------------------------------------------------- diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java index 7ec8172..432ad7e 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java @@ -33,6 +33,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.TimeZone; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -120,6 +121,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss private final Map> gauges = new HashMap<>(); private boolean logMissingCollection = true; + private String versionString; public MetricsHistoryHandler(String nodeName, MetricsHandler metricsHandler, SolrClient solrClient, SolrCloudManager cloudManager, int collectPeriod, int syncPeriod) { @@ -138,6 +140,14 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss gauges.put(Group.node.toString(), DEFAULT_NODE_GAUGES); gauges.put(Group.jvm.toString(), DEFAULT_JVM_GAUGES); + versionString = this.getClass().getPackage().getImplementationVersion(); + if (versionString == null) { + versionString = "?.?.?"; + } + if (versionString.length() > 24) { + versionString = versionString.substring(0, 24) + "..."; + } + collectService = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(1, new DefaultSolrThreadFactory("MetricsHistoryHandler")); collectService.setRemoveOnCancelPolicy(true); @@ -280,10 +290,11 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss // use AVERAGE consolidation, // use NaN when >50% samples are missing - def.addArchive(ConsolFun.AVERAGE, 0.5, 1, 120); // 2 hours + def.addArchive(ConsolFun.AVERAGE, 0.5, 1, 240); // 4 hours def.addArchive(ConsolFun.AVERAGE, 0.5, 10, 288); // 48 hours def.addArchive(ConsolFun.AVERAGE, 0.5, 60, 336); // 2 weeks def.addArchive(ConsolFun.AVERAGE, 0.5, 240, 180); // 2 months + def.addArchive(ConsolFun.AVERAGE, 0.5, 1440, 365); // 1 year return def; } @@ -299,15 +310,19 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss } public enum Cmd { - LIST, STATUS, GET, GRAPH, DELETE; + LIST, STATUS, GET, DELETE; static final Map actions = Collections.unmodifiableMap( Stream.of(Cmd.values()) - .collect(toMap(c -> c.name().toLowerCase(Locale.ROOT), Function.identity()))); + .collect(toMap(Cmd::toLower, Function.identity()))); public static Cmd get(String p) { return p == null ? null : actions.get(p.toLowerCase(Locale.ROOT)); } + + public String toLower() { + return toString().toLowerCase(Locale.ROOT); + } } public enum Format { @@ -315,11 +330,15 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss static final Map formats = Collections.unmodifiableMap( Stream.of(Format.values()) - .collect(toMap(c -> c.name().toLowerCase(Locale.ROOT), Function.identity()))); + .collect(toMap(Format::toLower, Function.identity()))); public static Format get(String p) { return p == null ? null : formats.get(p.toLowerCase(Locale.ROOT)); } + + public String toLower() { + return toString().toLowerCase(Locale.ROOT); + } } @@ -345,13 +364,17 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'name' is a required param"); } String[] dsNames = req.getParams().getParams("ds"); - Format format = Format.get(req.getParams().get("format", Format.LIST.toString()).toUpperCase(Locale.ROOT)); + String formatStr = req.getParams().get("format", Format.LIST.toString()); + Format format = Format.get(formatStr); + if (format == null) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "unknown 'format' param '" + formatStr + "', supported formats: " + Format.values()); + } if (!factory.exists(name)) { rsp.add("error", "'" + name + "' doesn't exist"); } else { // get a throwaway copy (safe to close and discard) RrdDb db = new RrdDb(URI_PREFIX + name, true, factory); - res = new NamedList(); + res = new NamedList<>(); NamedList data = new NamedList<>(); data.add("data", getData(db, dsNames, format, req.getParams())); ((NamedList)res).add(name, data); @@ -368,8 +391,10 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss } else { // get a throwaway copy (safe to close and discard) RrdDb db = new RrdDb(URI_PREFIX + name, true, factory); - Map map = new HashMap<>(); - map.put(name, Collections.singletonMap("status", reportStatus(db))); + NamedList map = new NamedList<>(); + NamedList status = new NamedList<>(); + status.add("status", reportStatus(db)); + map.add(name, status); db.close(); res = map; } @@ -392,16 +417,16 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss } } - private Map reportStatus(RrdDb db) throws IOException { - Map res = new LinkedHashMap<>(); - res.put("lastModified", db.getLastUpdateTime()); + private NamedList reportStatus(RrdDb db) throws IOException { + NamedList res = new SimpleOrderedMap<>(); + res.add("lastModified", db.getLastUpdateTime()); RrdDef def = db.getRrdDef(); - res.put("step", def.getStep()); - res.put("datasourceCount", db.getDsCount()); - res.put("archiveCount", db.getArcCount()); - res.put("datasourceNames", Arrays.asList(db.getDsNames())); + res.add("step", def.getStep()); + res.add("datasourceCount", db.getDsCount()); + res.add("archiveCount", db.getArcCount()); + res.add("datasourceNames", Arrays.asList(db.getDsNames())); List dss = new ArrayList<>(db.getDsCount()); - res.put("datasources", dss); + res.add("datasources", dss); for (DsDef dsDef : def.getDsDefs()) { Map map = new LinkedHashMap<>(); map.put("datasource", dsDef.dump()); @@ -410,7 +435,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss dss.add(map); } List archives = new ArrayList<>(db.getArcCount()); - res.put("archives", archives); + res.add("archives", archives); ArcDef[] arcDefs = def.getArcDefs(); for (int i = 0; i < db.getArcCount(); i++) { Archive a = db.getArchive(i); @@ -470,20 +495,27 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss switch (format) { case GRAPH: RrdGraphDef graphDef = new RrdGraphDef(); + graphDef.setTitle(name); graphDef.datasource(name, fd); graphDef.setStartTime(a.getStartTime() - a.getArcStep()); graphDef.setEndTime(a.getEndTime() + a.getArcStep()); + graphDef.setPoolUsed(false); graphDef.setAltAutoscale(true); graphDef.setAltYGrid(true); - graphDef.setShowSignature(false); + graphDef.setAltYMrtg(true); + graphDef.setSignature("Apache Solr " + versionString); + graphDef.setNoLegend(true); graphDef.setAntiAliasing(true); graphDef.setTextAntiAliasing(true); graphDef.setWidth(500); - graphDef.setHeight(125); + graphDef.setHeight(175); + graphDef.setTimeZone(TimeZone.getDefault()); + graphDef.setLocale(Locale.getDefault()); // redraw immediately graphDef.setLazy(false); - // "Solr" site background color - graphDef.line(name, new Color(0xD9, 0x41, 0x1E), name, 2); + // area with a border + graphDef.area(name, new Color(0xffb860), null); + graphDef.line(name, Color.RED, null, 1.0f); RrdGraph graph = new RrdGraph(graphDef); BufferedImage bi = new BufferedImage( graph.getRrdGraphInfo().getWidth(), http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/java/org/apache/solr/metrics/rrd/SolrRrdBackendFactory.java ---------------------------------------------------------------------- diff --git a/solr/core/src/java/org/apache/solr/metrics/rrd/SolrRrdBackendFactory.java b/solr/core/src/java/org/apache/solr/metrics/rrd/SolrRrdBackendFactory.java index a34aed3..4120d97 100644 --- a/solr/core/src/java/org/apache/solr/metrics/rrd/SolrRrdBackendFactory.java +++ b/solr/core/src/java/org/apache/solr/metrics/rrd/SolrRrdBackendFactory.java @@ -30,7 +30,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -38,21 +37,16 @@ import java.util.concurrent.TimeUnit; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.impl.CloudSolrClient; -import org.apache.solr.client.solrj.io.SolrClientCache; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.SolrCloseable; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrDocumentList; -import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.params.CollectionAdminParams; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.TimeSource; -import org.apache.solr.core.CoreContainer; import org.apache.solr.util.DefaultSolrThreadFactory; import org.rrd4j.core.RrdBackend; import org.rrd4j.core.RrdBackendFactory; @@ -60,7 +54,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * + * RRD backend factory using Solr documents as underlying storage. + *

RRD databases are identified by paths in the format solr:dbName. + * Typically the path will correspond to the name of metric or a group of metrics, eg: + * solr:QUERY./select.requests

+ *

NOTE: Solr doesn't register instances of this factory in the static + * registry {@link RrdBackendFactory#registerFactory(RrdBackendFactory)} because + * it's then impossible to manage its life-cycle.

*/ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrCloseable { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -88,6 +88,17 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos private final Map backends = new ConcurrentHashMap<>(); + /** + * Create a factory. + * @param nodeName node name. Documents are stored in a distributed collection and + * this parameter is needed to avoid namespace conflicts. + * @param solrClient SolrClient to use + * @param collection collection name where documents are stored (typicall this is + * {@link CollectionAdminParams#SYSTEM_COLL}) + * @param syncPeriod synchronization period in seconds - how often modified + * databases are stored as updated Solr documents + * @param timeSource time source + */ public SolrRrdBackendFactory(String nodeName, SolrClient solrClient, String collection, int syncPeriod, TimeSource timeSource) { this.nodeName = nodeName; this.solrClient = solrClient; @@ -145,6 +156,15 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos } } + /** + * Open (or get) a backend. + * @param path backend path (without URI scheme) + * @param readOnly if true then the backend will never be synchronized to Solr, + * and updates will be silently ignored. Read-only backends can + * be safely closed and discarded after use. + * @return an instance of Solr backend. + * @throws IOException on Solr error when retrieving existing data + */ @Override protected synchronized RrdBackend open(String path, boolean readOnly) throws IOException { ensureOpen(); @@ -200,6 +220,12 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos backends.remove(path); } + /** + * List all available databases created by this node name + * @param maxLength maximum number of results to return + * @return list of database names, or empty + * @throws IOException on server errors + */ public List list(int maxLength) throws IOException { Set names = new HashSet<>(); try { @@ -224,6 +250,10 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos return list; } + /** + * Remove all databases created by this node name. + * @throws IOException on server error + */ public void removeAll() throws IOException { for (Iterator it = backends.values().iterator(); it.hasNext(); ) { SolrRrdBackend backend = it.next(); @@ -241,6 +271,11 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos } } + /** + * Remove a database. + * @param path database path. + * @throws IOException on Solr exception + */ public void remove(String path) throws IOException { SolrRrdBackend backend = backends.get(path); if (backend != null) { @@ -254,7 +289,7 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos } } - public synchronized void maybeSyncBackends() { + synchronized void maybeSyncBackends() { if (closed) { return; } @@ -307,6 +342,13 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos } } + /** + * Check for existence of a backend. + * @param path backend path, without the URI scheme + * @return true when a backend exists. Note that a backend may exist only + * in memory if it was created recently within {@link #syncPeriod}. + * @throws IOException on Solr exception + */ @Override public boolean exists(String path) throws IOException { // check in-memory backends first http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/java/org/apache/solr/metrics/rrd/package-info.java ---------------------------------------------------------------------- diff --git a/solr/core/src/java/org/apache/solr/metrics/rrd/package-info.java b/solr/core/src/java/org/apache/solr/metrics/rrd/package-info.java new file mode 100644 index 0000000..2fd018c --- /dev/null +++ b/solr/core/src/java/org/apache/solr/metrics/rrd/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * This package contains {@link org.rrd4j.core.RrdBackendFactory} implementation + * that supports storing metrics history in Solr. + */ +package org.apache.solr.metrics.rrd; http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java index 26616c9..e349628 100644 --- a/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java @@ -1,6 +1,26 @@ +/* + * 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.cloud; +import javax.imageio.ImageIO; +import java.io.ByteArrayInputStream; +import java.io.IOException; import java.lang.invoke.MethodHandles; +import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.lucene.util.LuceneTestCase; @@ -12,11 +32,10 @@ import org.apache.solr.client.solrj.request.GenericSolrRequest; import org.apache.solr.cloud.autoscaling.sim.SimCloudManager; import org.apache.solr.common.params.CollectionAdminParams; import org.apache.solr.common.params.CommonParams; -import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.Base64; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.TimeSource; -import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.util.LogLevel; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -35,34 +54,28 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase { private static SolrCloudManager cloudManager; private static SolrClient solrClient; private static TimeSource timeSource; - private static SolrResourceLoader loader; - - private static int SPEED; @BeforeClass public static void setupCluster() throws Exception { - boolean simulated = random().nextBoolean() && false; + boolean simulated = random().nextBoolean(); if (simulated) { - SPEED = 50; - cloudManager = SimCloudManager.createCluster(2, TimeSource.get("simTime:" + SPEED)); + cloudManager = SimCloudManager.createCluster(1, TimeSource.get("simTime:50")); solrClient = ((SimCloudManager)cloudManager).simGetSolrClient(); - loader = ((SimCloudManager) cloudManager).getLoader(); } - configureCluster(2) + configureCluster(1) .addConfig("conf", configset("cloud-minimal")) .configure(); if (!simulated) { cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager(); solrClient = cluster.getSolrClient(); - loader = cluster.getJettySolrRunner(0).getCoreContainer().getResourceLoader(); - SPEED = 1; } timeSource = cloudManager.getTimeSource(); // create .system - CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 2) + CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 1) .process(solrClient); CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL, - 30, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 2)); + 30, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 1)); + solrClient.query(CollectionAdminParams.SYSTEM_COLL, params(CommonParams.Q, "*:*")); // sleep a little to allow the handler to collect some metrics timeSource.sleep(90000); } @@ -80,12 +93,33 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase { public void testList() throws Exception { NamedList rsp = solrClient.request(createHistoryRequest(params(CommonParams.ACTION, "list"))); assertNotNull(rsp); - log.info("Response: " + rsp); + // expected solr.jvm, solr.node and solr.core..system replica 1 + List lst = (List)rsp.get("metrics"); + assertNotNull(lst); + assertEquals(lst.toString(), 3, lst.size()); + assertTrue(lst.toString(), lst.contains("solr.jvm")); + assertTrue(lst.toString(), lst.contains("solr.node")); + assertTrue(lst.toString(), lst.contains("solr.core..system.shard1.replica_n1")); } @Test public void testStatus() throws Exception { - + NamedList rsp = solrClient.request(createHistoryRequest( + params(CommonParams.ACTION, "status", CommonParams.NAME, "solr.jvm"))); + assertNotNull(rsp); + NamedList map = (NamedList)rsp.get("metrics"); + assertEquals(map.toString(), 1, map.size()); + map = (NamedList)map.get("solr.jvm"); + assertNotNull(map); + NamedList status = (NamedList)map.get("status"); + assertNotNull(status); + assertEquals(status.toString(), 7, status.size()); + List lst = (List)status.get("datasources"); + assertNotNull(lst); + assertEquals(lst.toString(), 3, lst.size()); + lst = (List)status.get("archives"); + assertNotNull(lst); + assertEquals(lst.toString(), 5, lst.size()); } @Test @@ -93,7 +127,63 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase { NamedList rsp = solrClient.request(createHistoryRequest(params( CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm"))); assertNotNull(rsp); - log.info("Response: " + rsp); + // default format is LIST + NamedList data = (NamedList)rsp.findRecursive("metrics", "solr.jvm", "data"); + assertNotNull(data); + data.forEach((k, v) -> { + NamedList entry = (NamedList)v; + List lst = entry.getAll("timestamps"); + assertNotNull(lst); + assertTrue("timestamps", lst.size() > 0); + // 3 metrics, so the total size of values is 3 * the size of timestamps + entry = (NamedList)entry.get("values"); + assertNotNull(entry); + assertEquals(lst.size() * 3, entry.size()); + }); + + // get STRING + rsp = solrClient.request(createHistoryRequest(params( + CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm", "format", "string"))); + data = (NamedList)rsp.findRecursive("metrics", "solr.jvm", "data"); + assertNotNull(data); + data.forEach((k, v) -> { + NamedList entry = (NamedList)v; + List lst = entry.getAll("timestamps"); + assertNotNull(lst); + assertEquals("timestamps", 1, lst.size()); + String timestampString = (String)lst.get(0); + String[] timestamps = timestampString.split(("\n")); + assertTrue(timestampString, timestamps.length > 1); + entry = (NamedList)entry.get("values"); + assertNotNull(entry); + assertEquals(3, entry.size()); + entry.forEach((vk, vv) -> { + String valString = (String)vv; + String[] values = valString.split("\n"); + assertEquals(valString, timestamps.length, values.length); + }); + }); + + // get GRAPH + rsp = solrClient.request(createHistoryRequest(params( + CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm", "format", "graph"))); + data = (NamedList)rsp.findRecursive("metrics", "solr.jvm", "data"); + assertNotNull(data); + data.forEach((k, v) -> { + NamedList entry = (NamedList) v; + entry = (NamedList)entry.get("values"); + assertNotNull(entry); + assertEquals(3, entry.size()); + entry.forEach((vk, vv) -> { + String valString = (String)vv; + byte[] img = Base64.base64ToByteArray(valString); + try { + ImageIO.read(new ByteArrayInputStream(img)); + } catch (IOException e) { + fail("Error reading image data: " + e.toString()); + } + }); + }); } public static SolrRequest createHistoryRequest(SolrParams params) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java index d8065bc..d7bb5b1 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java @@ -226,6 +226,7 @@ public class SimCloudManager implements SolrCloudManager { MetricsHandler metricsHandler = new MetricsHandler(metricManager); this.historyHandler = new MetricsHistoryHandler("1.0.0.1:1111_solr", metricsHandler, solrClient, this, MetricsHistoryHandler.DEFAULT_COLLECT_PERIOD, SolrRrdBackendFactory.DEFAULT_SYNC_PERIOD); + this.historyHandler.initializeMetrics(metricManager, SolrMetricManager.getRegistryName(SolrInfoBean.Group.node), metricTag, CommonParams.METRICS_HISTORY_PATH); triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers"); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java index 85dcb68..4c23cc4 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java @@ -425,6 +425,13 @@ public class SimClusterStateProvider implements ClusterStateProvider { disk = SimCloudManager.DEFAULT_DISK; } cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk - 1); + // fake metrics + String registry = SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, replicaInfo.getCollection(), + replicaInfo.getShard(), + Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCore())); + cloudManager.getMetricManager().registry(registry).counter("UPDATE./update.requests"); + cloudManager.getMetricManager().registry(registry).counter("QUERY./select.requests"); + cloudManager.getMetricManager().registerGauge(null, registry, () -> 1000, "", true, "INDEX.sizeInBytes"); if (runLeaderElection) { simRunLeaderElection(Collections.singleton(replicaInfo.getCollection()), true); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java index c012890..e8e50ed 100644 --- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java @@ -40,7 +40,7 @@ import org.rrd4j.core.RrdDb; /** * */ -@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG") +@LogLevel("org.apache.solr.cloud=DEBUG") public class MetricsHistoryHandlerTest extends SolrCloudTestCase { private static SolrCloudManager cloudManager; @@ -112,7 +112,8 @@ public class MetricsHistoryHandlerTest extends SolrCloudTestCase { int dsCount = db.getDsCount(); int arcCount = db.getArcCount(); assertTrue("dsCount should be > 0, was " + dsCount, dsCount > 0); - assertEquals("arcCount", 4, arcCount); + assertEquals("arcCount", 5, arcCount); + db.close(); } } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java index 39c1125..a4fd67f 100644 --- a/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java +++ b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java @@ -160,6 +160,8 @@ public class SolrRrdBackendFactoryTest extends SolrTestCaseJ4 { assertEquals(100.0, two[i], 0.00001); } + db.close(); + // open a read-only version of the db RrdDb readOnly = new RrdDb("solr:foo", true, factory); s = readOnly.createSample(); @@ -175,6 +177,7 @@ public class SolrRrdBackendFactoryTest extends SolrTestCaseJ4 { assertTrue(newDoc.toString(), newDoc.equals(doc)); newTimestamp = ((Date)newDoc.getFieldValue("timestamp")).getTime(); assertEquals(newTimestamp, timestamp); + readOnly.close(); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java ---------------------------------------------------------------------- diff --git a/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java b/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java index ae672d7..11e58c3 100644 --- a/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java +++ b/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java @@ -17,7 +17,6 @@ package org.apache.solr.util; import java.io.IOException; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/licenses/rrd4j-3.2.jar.sha1 ---------------------------------------------------------------------- diff --git a/solr/licenses/rrd4j-3.2.jar.sha1 b/solr/licenses/rrd4j-3.2.jar.sha1 new file mode 100644 index 0000000..5304919 --- /dev/null +++ b/solr/licenses/rrd4j-3.2.jar.sha1 @@ -0,0 +1 @@ +8d480d5aa87b3d358862b78d6fa3660396220dc7 http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/licenses/rrd4j-LICENSE-ASL.txt ---------------------------------------------------------------------- diff --git a/solr/licenses/rrd4j-LICENSE-ASL.txt b/solr/licenses/rrd4j-LICENSE-ASL.txt new file mode 100644 index 0000000..7a4a3ea --- /dev/null +++ b/solr/licenses/rrd4j-LICENSE-ASL.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. \ No newline at end of file http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2841f442/solr/licenses/rrd4j-NOTICE.txt ---------------------------------------------------------------------- diff --git a/solr/licenses/rrd4j-NOTICE.txt b/solr/licenses/rrd4j-NOTICE.txt new file mode 100644 index 0000000..841dd30 --- /dev/null +++ b/solr/licenses/rrd4j-NOTICE.txt @@ -0,0 +1,2 @@ +GitHub: https://github.com/rrd4j/rrd4j/ +Maven-generated site: http://rrd4j.org/site/