From 57287c8520e51c10531cb7e5d6c46e7a43b27a3c Mon Sep 17 00:00:00 2001 From: Steve Woods Date: Mon, 29 Jul 2024 06:09:44 -0400 Subject: [PATCH 01/38] SOLR-17322: Make RankQuery.getTopDocsCollector use covariant generic types (#2506) --- solr/CHANGES.txt | 3 + .../solr/search/AbstractReRankQuery.java | 2 +- .../solr/search/ExportQParserPlugin.java | 2 +- .../org/apache/solr/search/RankQuery.java | 2 +- .../org/apache/solr/search/RankQueryTest.java | 71 +++++++++++++++++++ .../solr/search/RankQueryTestPlugin.java | 2 +- .../solr/search/SolrIndexSearcherTest.java | 2 +- 7 files changed, 79 insertions(+), 5 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 1077e653d98..ea7a5b7b71c 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -240,6 +240,9 @@ Other Changes * SOLR-15831: Refactor bin/solr and bin/solr.cmd commands integration with AuthTool, DeleteTool, and PackageTool to delegate arg parsing to Java code. Removed limitation of PackageTool only being executed on an active Solr node. (Eric Pugh) +* SOLR-17322: Once again allow rank queries to use custom TopDocsCollectors that operate on types that extend + ScoreDocs (covariant generic types) broken in Solr 9.0. (Stephen Woods via Christine Poerschke) + ================== 9.6.1 ================== Bug Fixes --------------------- diff --git a/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java b/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java index f2eb6de2c3b..62c4bd98bbc 100644 --- a/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java +++ b/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java @@ -75,7 +75,7 @@ public MergeStrategy getMergeStrategy() { @Override @SuppressWarnings({"unchecked"}) - public TopDocsCollector getTopDocsCollector( + public TopDocsCollector getTopDocsCollector( int len, QueryCommand cmd, IndexSearcher searcher) throws IOException { if (this.boostedPriority == null) { SolrRequestInfo info = SolrRequestInfo.getRequestInfo(); diff --git a/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java index e43f229d8a5..8529533dfb3 100644 --- a/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java @@ -104,7 +104,7 @@ public Query rewrite(IndexReader reader) throws IOException { } @Override - public TopDocsCollector getTopDocsCollector( + public TopDocsCollector getTopDocsCollector( int len, QueryCommand cmd, IndexSearcher searcher) throws IOException { int leafCount = searcher.getTopReaderContext().leaves().size(); FixedBitSet[] sets = new FixedBitSet[leafCount]; diff --git a/solr/core/src/java/org/apache/solr/search/RankQuery.java b/solr/core/src/java/org/apache/solr/search/RankQuery.java index ec6ae26f5a3..3f281ebbd29 100644 --- a/solr/core/src/java/org/apache/solr/search/RankQuery.java +++ b/solr/core/src/java/org/apache/solr/search/RankQuery.java @@ -29,7 +29,7 @@ */ public abstract class RankQuery extends ExtendedQueryBase { - public abstract TopDocsCollector getTopDocsCollector( + public abstract TopDocsCollector getTopDocsCollector( int len, QueryCommand cmd, IndexSearcher searcher) throws IOException; public abstract MergeStrategy getMergeStrategy(); diff --git a/solr/core/src/test/org/apache/solr/search/RankQueryTest.java b/solr/core/src/test/org/apache/solr/search/RankQueryTest.java index 8bcdea1d50c..f9c3a8fa043 100644 --- a/solr/core/src/test/org/apache/solr/search/RankQueryTest.java +++ b/solr/core/src/test/org/apache/solr/search/RankQueryTest.java @@ -16,8 +16,19 @@ */ package org.apache.solr.search; +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.util.PriorityQueue; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.handler.component.MergeStrategy; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -106,4 +117,64 @@ public void testPluggableCollector() { "//result/doc[2]/str[@name='id'][.='6']", "//result/doc[1]/str[@name='id'][.='5']"); } + + // The following static classes are intended to ensure that support of covariant + // ScoreDocs is supported in rank queries. MyRankQuery will fail to compile + // if covariant ScoreDocs are not supported because it returns a TopDocsCollector + // for MyScoreDoc (a subtype of ScoreDoc). + static class MyScoreDoc extends ScoreDoc { + public int someOtherField; + + public MyScoreDoc(int doc, float score, int shardIndex, int someOtherField) { + super(doc, score, shardIndex); + this.someOtherField = someOtherField; + } + } + + static class MyTopDocsCollector extends TopDocsCollector { + public MyTopDocsCollector(PriorityQueue pq) { + super(pq); + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return null; + } + } + + static class MyRankQuery extends RankQuery { + @Override + public TopDocsCollector getTopDocsCollector( + int len, QueryCommand cmd, IndexSearcher searcher) throws IOException { + return new MyTopDocsCollector(null); + } + + @Override + public MergeStrategy getMergeStrategy() { + return null; + } + + @Override + public RankQuery wrap(Query mainQuery) { + return this; + } + + @Override + public void visit(QueryVisitor visitor) {} + + @Override + public int hashCode() { + return 1; + } + + @Override + public boolean equals(Object obj) { + return true; + } + } } diff --git a/solr/core/src/test/org/apache/solr/search/RankQueryTestPlugin.java b/solr/core/src/test/org/apache/solr/search/RankQueryTestPlugin.java index b11728e0283..c82661200b3 100644 --- a/solr/core/src/test/org/apache/solr/search/RankQueryTestPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/RankQueryTestPlugin.java @@ -140,7 +140,7 @@ public TestRankQuery(int collector, int mergeStrategy) { } @Override - public TopDocsCollector getTopDocsCollector( + public TopDocsCollector getTopDocsCollector( int len, QueryCommand cmd, IndexSearcher searcher) { if (collector == 0) return new TestCollector(null); else return new TestCollector1(null); diff --git a/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java index be7f458587b..49fcc21e45e 100644 --- a/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java +++ b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java @@ -327,7 +327,7 @@ public String toString(String field) { } @Override - public TopDocsCollector getTopDocsCollector( + public TopDocsCollector getTopDocsCollector( int len, QueryCommand cmd, IndexSearcher searcher) throws IOException { return new ReRankCollector( len, From e00cc7b1e4115d86cd8d57fb29a719ac1aa9e640 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Mon, 29 Jul 2024 11:10:36 +0100 Subject: [PATCH 02/38] SOLR-17337: Show proper distributed stage id (#2594) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Torsten Bøgh Köster --- solr/CHANGES.txt | 2 ++ .../solr/handler/component/DebugComponent.java | 17 +++++++++++++++-- .../handler/component/DebugComponentTest.java | 10 ++++++++++ 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index ea7a5b7b71c..5676087acec 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -212,6 +212,8 @@ Bug Fixes * SOLR-17369: Fix "flags" usage in FunctionQParser that caused some issues in vectorSimilarity() with BYTE vector constants (hossman) +* SOLR-17337: Display all custom distributed stages in debug output. (Torsten Bøgh Köster, Christine Poerschke) + Dependency Upgrades --------------------- (No changes) diff --git a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java index 0565290cf06..001b6c3338a 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java @@ -19,6 +19,7 @@ import static org.apache.solr.common.params.CommonParams.FQ; import static org.apache.solr.common.params.CommonParams.JSON; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.lang.reflect.Array; import java.util.ArrayList; @@ -181,16 +182,28 @@ public void modifyRequest(ResponseBuilder rb, SearchComponent who, ShardRequest } } + @VisibleForTesting + protected String getDistributedStageName(int stage) { + String stageName = stages.get(stage); + + if (stageName == null) { + stageName = "STAGE_" + Integer.toString(stage); + } + + return stageName; + } + @Override public void handleResponses(ResponseBuilder rb, ShardRequest sreq) { if (rb.isDebugTrack() && rb.isDistrib && !rb.finished.isEmpty()) { @SuppressWarnings("unchecked") NamedList stageList = (NamedList) - ((NamedList) rb.getDebugInfo().get("track")).get(stages.get(rb.stage)); + ((NamedList) rb.getDebugInfo().get("track")) + .get(getDistributedStageName(rb.stage)); if (stageList == null) { stageList = new SimpleOrderedMap<>(); - rb.addDebug(stageList, "track", stages.get(rb.stage)); + rb.addDebug(stageList, "track", getDistributedStageName(rb.stage)); } for (ShardResponse response : sreq.responses) { stageList.add(response.getShard(), getTrackResponse(response)); diff --git a/solr/core/src/test/org/apache/solr/handler/component/DebugComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/DebugComponentTest.java index dcc97e7fe2a..9778a4bb078 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/DebugComponentTest.java +++ b/solr/core/src/test/org/apache/solr/handler/component/DebugComponentTest.java @@ -297,4 +297,14 @@ private void addRequestId(ResponseBuilder rb, String requestId) { params.add(CommonParams.REQUEST_ID, requestId); rb.req.setParams(params); } + + @Test + public void testDistributedStageResolution() throws IOException { + final DebugComponent debugComponent = new DebugComponent(); + assertEquals( + "PARSE_QUERY", debugComponent.getDistributedStageName(ResponseBuilder.STAGE_PARSE_QUERY)); + assertEquals("DONE", debugComponent.getDistributedStageName(ResponseBuilder.STAGE_DONE)); + assertEquals("STAGE_400", debugComponent.getDistributedStageName(400)); + assertEquals("STAGE_10000", debugComponent.getDistributedStageName(10000)); + } } From 57fa8b968b1f6b82e048362a62906cda1bc153e7 Mon Sep 17 00:00:00 2001 From: aparnasuresh85 Date: Mon, 29 Jul 2024 09:29:06 -0400 Subject: [PATCH 03/38] Refactor: Move Collection Properties backend out of ZkStateReader (#2585) ZkStateReader now delegates to CollectionPropertiesZkStateReader for this functionality. No API change. No functional / observable change. ZkStateReader has been doing too much! --- .../CollectionPropertiesZkStateReader.java | 413 ++++++++++++++++++ .../solr/common/cloud/ZkStateReader.java | 349 +-------------- 2 files changed, 430 insertions(+), 332 deletions(-) create mode 100644 solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/CollectionPropertiesZkStateReader.java diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/CollectionPropertiesZkStateReader.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/CollectionPropertiesZkStateReader.java new file mode 100644 index 00000000000..2bb36116afd --- /dev/null +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/CollectionPropertiesZkStateReader.java @@ -0,0 +1,413 @@ +/* + * 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.common.cloud; + +import static java.util.Collections.emptyMap; + +import java.io.Closeable; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.ZkStateReader.CollectionWatch; +import org.apache.solr.common.util.ExecutorUtil; +import org.apache.solr.common.util.SolrNamedThreadFactory; +import org.apache.solr.common.util.Utils; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Fetches and manages collection properties from a ZooKeeper ensemble */ +public class CollectionPropertiesZkStateReader implements Closeable { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private volatile boolean closed = false; + + private final SolrZkClient zkClient; + private final ZkStateReader zkStateReader; + + /** Collection properties being actively watched */ + private final ConcurrentHashMap watchedCollectionProps = + new ConcurrentHashMap<>(); + + /** + * Manages ZooKeeper watchers for each collection. These watchers monitor changes to the + * properties of the collection in ZooKeeper. When a change is detected in ZooKeeper, the watcher + * triggers an update, which then notifies the relevant "collectionPropsObserver". + */ + private final ConcurrentHashMap collectionPropsWatchers = + new ConcurrentHashMap<>(); + + /** + * Manages a list of observers (listeners) for each collection. These observers need to be + * notified when the properties of the collection change. When a collection's properties change, + * all registered observers for that collection are notified by a "collectionPropWatcher". + */ + private ConcurrentHashMap> + collectionPropsObservers = new ConcurrentHashMap<>(); + + /** Used to submit notifications to Collection Properties watchers in order */ + private final ExecutorService collectionPropsNotifications = + ExecutorUtil.newMDCAwareSingleThreadExecutor( + new SolrNamedThreadFactory("collectionPropsNotifications")); + + private final ExecutorService notifications = + ExecutorUtil.newMDCAwareCachedThreadPool("cachecleaner"); + + // only kept to identify if the cleaner has already been started. + private Future collectionPropsCacheCleaner; + + public CollectionPropertiesZkStateReader(ZkStateReader zkStateReader) { + this.zkClient = zkStateReader.getZkClient(); + this.zkStateReader = zkStateReader; + } + + /** + * Get and cache collection properties for a given collection. If the collection is watched, or + * still cached simply return it from the cache, otherwise fetch it directly from zookeeper and + * retain the value for at least cacheForMillis milliseconds. Cached properties are watched in + * zookeeper and updated automatically. This version of {@code getCollectionProperties} should be + * used when properties need to be consulted frequently in the absence of an active {@link + * CollectionPropsWatcher}. + * + * @param collection The collection for which properties are desired + * @param cacheForMillis The minimum number of milliseconds to maintain a cache for the specified + * collection's properties. Setting a {@code CollectionPropsWatcher} will override this value + * and retain the cache for the life of the watcher. A lack of changes in zookeeper may allow + * the caching to remain for a greater duration up to the cycle time of {@code CacheCleaner}. + * Passing zero for this value will explicitly remove the cached copy if and only if it is due + * to expire and no watch exists. Any positive value will extend the expiration time if + * required. + * @return a map representing the key/value properties for the collection. + */ + public Map getCollectionProperties(final String collection, long cacheForMillis) { + synchronized (watchedCollectionProps) { // synchronized on the specific collection + Watcher watcher = null; + if (cacheForMillis > 0) { + watcher = + collectionPropsWatchers.compute( + collection, + (c, w) -> + w == null ? new PropsWatcher(c, cacheForMillis) : w.renew(cacheForMillis)); + } + VersionedCollectionProps vprops = watchedCollectionProps.get(collection); + boolean haveUnexpiredProps = vprops != null && vprops.cacheUntilNs > System.nanoTime(); + long untilNs = + System.nanoTime() + TimeUnit.NANOSECONDS.convert(cacheForMillis, TimeUnit.MILLISECONDS); + Map properties; + if (haveUnexpiredProps) { + properties = vprops.props; + vprops.cacheUntilNs = Math.max(vprops.cacheUntilNs, untilNs); + } else { + try { + VersionedCollectionProps vcp = fetchCollectionProperties(collection, watcher); + properties = vcp.props; + if (cacheForMillis > 0) { + vcp.cacheUntilNs = untilNs; + watchedCollectionProps.put(collection, vcp); + } else { + // we're synchronized on watchedCollectionProps and we can only get here if we have + // found an expired vprops above, so it is safe to remove the cached value and let the + // GC free up some mem a bit sooner. + if (!collectionPropsObservers.containsKey(collection)) { + watchedCollectionProps.remove(collection); + } + } + } catch (Exception e) { + throw new SolrException( + SolrException.ErrorCode.SERVER_ERROR, + "Error reading collection properties", + SolrZkClient.checkInterrupted(e)); + } + } + return properties; + } + } + + @Override + public void close() { + this.closed = true; + notifications.shutdownNow(); + ExecutorUtil.shutdownAndAwaitTermination(notifications); + ExecutorUtil.shutdownAndAwaitTermination(collectionPropsNotifications); + } + + private static class VersionedCollectionProps { + int zkVersion; + Map props; + long cacheUntilNs = 0; + + VersionedCollectionProps(int zkVersion, Map props) { + this.zkVersion = zkVersion; + this.props = props; + } + } + + /** Watches collection properties */ + class PropsWatcher implements Watcher { + private final String coll; + private long watchUntilNs; + + PropsWatcher(String coll) { + this.coll = coll; + watchUntilNs = 0; + } + + PropsWatcher(String coll, long forMillis) { + this.coll = coll; + watchUntilNs = + System.nanoTime() + TimeUnit.NANOSECONDS.convert(forMillis, TimeUnit.MILLISECONDS); + } + + public PropsWatcher renew(long forMillis) { + watchUntilNs = + System.nanoTime() + TimeUnit.NANOSECONDS.convert(forMillis, TimeUnit.MILLISECONDS); + return this; + } + + @Override + public void process(WatchedEvent event) { + // session events are not change events, and do not remove the watcher + if (Event.EventType.None.equals(event.getType())) { + return; + } + + boolean expired = System.nanoTime() > watchUntilNs; + if (!collectionPropsObservers.containsKey(coll) && expired) { + // No one can be notified of the change, we can ignore it and "unset" the watch + log.debug("Ignoring property change for collection {}", coll); + return; + } + + log.info( + "A collection property change: [{}] for collection [{}] has occurred - updating...", + event, + coll); + + refreshAndWatch(true); + } + + /** + * Refresh collection properties from ZK and leave a watch for future changes. Updates the + * properties in watchedCollectionProps with the results of the refresh. Optionally notifies + * watchers + */ + void refreshAndWatch(boolean notifyWatchers) { + try { + synchronized (watchedCollectionProps) { // making decisions based on the result of a get... + VersionedCollectionProps vcp = fetchCollectionProperties(coll, this); + Map properties = vcp.props; + VersionedCollectionProps existingVcp = watchedCollectionProps.get(coll); + if (existingVcp == null + || // never called before, record what we found + vcp.zkVersion > existingVcp.zkVersion + || // newer info we should update + vcp.zkVersion == -1) { // node was deleted start over + watchedCollectionProps.put(coll, vcp); + if (notifyWatchers) { + notifyPropsWatchers(coll, properties); + } + if (vcp.zkVersion == -1 && existingVcp != null) { // Collection DELETE detected + + // We should not be caching a collection that has been deleted. + watchedCollectionProps.remove(coll); + + // core ref counting not relevant here, don't need canRemove(), we just sent + // a notification of an empty set of properties, no reason to watch what doesn't + // exist. + collectionPropsObservers.remove(coll); + + // This is the one time we know it's safe to throw this out. We just failed to set the + // watch due to an NoNodeException, so it isn't held by ZK and can't re-set itself due + // to an update. + collectionPropsWatchers.remove(coll); + } + } + } + } catch (KeeperException.SessionExpiredException + | KeeperException.ConnectionLossException e) { + log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: ", e); + } catch (KeeperException e) { + log.error("Lost collection property watcher for {} due to ZK error", coll, e); + throw new ZooKeeperException( + SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.error( + "Lost collection property watcher for {} due to the thread being interrupted", coll, e); + } + } + } + + public void registerCollectionPropsWatcher( + final String collection, CollectionPropsWatcher propsWatcher) { + AtomicBoolean watchSet = new AtomicBoolean(false); + collectionPropsObservers.compute( + collection, + (k, v) -> { + if (v == null) { + v = new CollectionWatch<>(); + watchSet.set(true); + } + v.stateWatchers.add(propsWatcher); + return v; + }); + + if (watchSet.get()) { + collectionPropsWatchers.computeIfAbsent(collection, PropsWatcher::new).refreshAndWatch(false); + } + } + + protected void refreshCollectionProperties() { + collectionPropsObservers.forEach( + (k, v) -> { + collectionPropsWatchers.computeIfAbsent(k, PropsWatcher::new).refreshAndWatch(true); + }); + } + + public static String getCollectionPropsPath(final String collection) { + return ZkStateReader.COLLECTIONS_ZKNODE + + '/' + + collection + + '/' + + ZkStateReader.COLLECTION_PROPS_ZKNODE; + } + + private VersionedCollectionProps fetchCollectionProperties(String collection, Watcher watcher) + throws KeeperException, InterruptedException { + final String znodePath = getCollectionPropsPath(collection); + // lazy init cache cleaner once we know someone is using collection properties. + if (collectionPropsCacheCleaner == null) { + synchronized (zkStateReader.getUpdateLock()) { // Double-checked locking + if (collectionPropsCacheCleaner == null) { + collectionPropsCacheCleaner = notifications.submit(new CacheCleaner()); + } + } + } + while (true) { + try { + Stat stat = new Stat(); + byte[] data = zkClient.getData(znodePath, watcher, stat, true); + @SuppressWarnings("unchecked") + Map props = (Map) Utils.fromJSON(data); + return new VersionedCollectionProps(stat.getVersion(), props); + } catch (ClassCastException e) { + throw new SolrException( + SolrException.ErrorCode.SERVER_ERROR, + "Unable to parse collection properties for collection " + collection, + e); + } catch (KeeperException.NoNodeException e) { + if (watcher != null) { + // Leave an exists watch in place in case a collectionprops.json is created later. + Stat exists = zkClient.exists(znodePath, watcher, true); + if (exists != null) { + // Rare race condition, we tried to fetch the data and couldn't find it, then we found + // it exists. Loop and try again. + continue; + } + } + return new VersionedCollectionProps(-1, emptyMap()); + } + } + } + + private void notifyPropsWatchers(String collection, Map properties) { + try { + collectionPropsNotifications.submit(new PropsNotification(collection, properties)); + } catch (RejectedExecutionException e) { + if (!closed) { + log.error("Couldn't run collection properties notifications for {}", collection, e); + } + } + } + + private class PropsNotification implements Runnable { + + private final String collection; + private final Map collectionProperties; + private final List watchers = new ArrayList<>(); + + private PropsNotification(String collection, Map collectionProperties) { + this.collection = collection; + this.collectionProperties = collectionProperties; + // guarantee delivery of notification regardless of what happens to collectionPropsObservers + // while we wait our turn in the executor by capturing the list on creation. + collectionPropsObservers.compute( + collection, + (k, v) -> { + if (v == null) return null; + watchers.addAll(v.stateWatchers); + return v; + }); + } + + @Override + public void run() { + for (CollectionPropsWatcher watcher : watchers) { + if (watcher.onStateChanged(collectionProperties)) { + removeCollectionPropsWatcher(collection, watcher); + } + } + } + } + + private class CacheCleaner implements Runnable { + @Override + public void run() { + while (!Thread.interrupted()) { + try { + Thread.sleep(60000); + } catch (InterruptedException e) { + // Executor shutdown will send us an interrupt + break; + } + watchedCollectionProps + .entrySet() + .removeIf( + entry -> + entry.getValue().cacheUntilNs < System.nanoTime() + && !collectionPropsObservers.containsKey(entry.getKey())); + } + } + } + + public void removeCollectionPropsWatcher(String collection, CollectionPropsWatcher watcher) { + collectionPropsObservers.compute( + collection, + (k, v) -> { + if (v == null) return null; + v.stateWatchers.remove(watcher); + if (v.canBeRemoved()) { + // don't want this to happen in middle of other blocks that might add it back. + synchronized (watchedCollectionProps) { + watchedCollectionProps.remove(collection); + } + return null; + } + return v; + }); + } +} diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java index b9882ddcc11..c2f76d62de3 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java @@ -37,7 +37,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -58,7 +57,6 @@ import org.apache.solr.common.util.CommonTestInjection; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.ObjectReleaseTracker; -import org.apache.solr.common.util.SolrNamedThreadFactory; import org.apache.solr.common.util.Utils; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -156,6 +154,7 @@ public class ZkStateReader implements SolrCloseable { public static final String CONTAINER_PLUGINS = "plugin"; public static final String PLACEMENT_PLUGIN = "placement-plugin"; + private final CollectionPropertiesZkStateReader collectionPropertiesZkStateReader; /** A view of the current state of all collections. */ protected volatile ClusterState clusterState; @@ -173,14 +172,6 @@ public class ZkStateReader implements SolrCloseable { private final ConcurrentHashMap lazyCollectionStates = new ConcurrentHashMap<>(); - /** Collection properties being actively watched */ - private final ConcurrentHashMap watchedCollectionProps = - new ConcurrentHashMap<>(); - - /** Watchers of Collection properties */ - private final ConcurrentHashMap collectionPropsWatchers = - new ConcurrentHashMap<>(); - private volatile SortedSet liveNodes = emptySortedSet(); private volatile Map clusterProperties = Collections.emptyMap(); @@ -191,11 +182,6 @@ public class ZkStateReader implements SolrCloseable { */ private DocCollectionWatches collectionWatches = new DocCollectionWatches(); - // named this observers so there's less confusion between CollectionPropsWatcher map and the - // PropsWatcher map. - private ConcurrentHashMap> - collectionPropsObservers = new ConcurrentHashMap<>(); - private Set cloudCollectionsListeners = ConcurrentHashMap.newKeySet(); private final ExecutorService notifications = ExecutorUtil.newMDCAwareCachedThreadPool("watches"); @@ -204,17 +190,9 @@ public class ZkStateReader implements SolrCloseable { private Set clusterPropertiesListeners = ConcurrentHashMap.newKeySet(); - /** Used to submit notifications to Collection Properties watchers in order */ - private final ExecutorService collectionPropsNotifications = - ExecutorUtil.newMDCAwareSingleThreadExecutor( - new SolrNamedThreadFactory("collectionPropsNotifications")); - private static final long LAZY_CACHE_TIME = TimeUnit.NANOSECONDS.convert(STATE_UPDATE_DELAY, TimeUnit.MILLISECONDS); - // only kept to identify if the cleaner has already been started. - private Future collectionPropsCacheCleaner; - /** * Gets the ZkStateReader inside a ZK based SolrClient. * @@ -229,7 +207,7 @@ public static ZkStateReader from(CloudSolrClient solrClient) { } } - private static class CollectionWatch { + protected static class CollectionWatch { int coreRefCount = 0; Set stateWatchers = ConcurrentHashMap.newKeySet(); @@ -417,6 +395,7 @@ public ZkStateReader(SolrZkClient zkClient, Runnable securityNodeListener) { this.zkClient = zkClient; this.closeClient = false; this.securityNodeWatcher = new SecurityNodeWatcher(this, securityNodeListener); + collectionPropertiesZkStateReader = new CollectionPropertiesZkStateReader(this); assert ObjectReleaseTracker.track(this); } @@ -454,7 +433,7 @@ public ZkStateReader( this.zkClient = builder.build(); this.closeClient = true; this.securityNodeWatcher = null; - + collectionPropertiesZkStateReader = new CollectionPropertiesZkStateReader(this); assert ObjectReleaseTracker.track(this); } @@ -592,11 +571,7 @@ public synchronized void createClusterStateWatchersAndUpdate() if (securityNodeWatcher != null) { securityNodeWatcher.register(); } - - collectionPropsObservers.forEach( - (k, v) -> { - collectionPropsWatchers.computeIfAbsent(k, PropsWatcher::new).refreshAndWatch(true); - }); + collectionPropertiesZkStateReader.refreshCollectionProperties(); } catch (KeeperException.NoNodeException nne) { throw new SolrException( ErrorCode.SERVICE_UNAVAILABLE, @@ -896,6 +871,10 @@ public Object getUpdateLock() { return this; } + public SolrZkClient getZKClient() { + return zkClient; + } + @Override public void close() { this.closed = true; @@ -909,7 +888,7 @@ public void close() { }); ExecutorUtil.shutdownAndAwaitTermination(notifications); - ExecutorUtil.shutdownAndAwaitTermination(collectionPropsNotifications); + collectionPropertiesZkStateReader.close(); if (closeClient) { zkClient.close(); } @@ -1190,131 +1169,18 @@ private void loadClusterProperties() { } } - /** - * Get collection properties for a given collection. If the collection is watched, simply return - * it from the cache, otherwise fetch it directly from zookeeper. This is a convenience for {@code - * getCollectionProperties(collection,0)} - * - * @param collection the collection for which properties are desired - * @return a map representing the key/value properties for the collection. - */ + /** Get properties for a specific collection */ public Map getCollectionProperties(final String collection) { - return getCollectionProperties(collection, 0); + return collectionPropertiesZkStateReader.getCollectionProperties(collection, 0); } - /** - * Get and cache collection properties for a given collection. If the collection is watched, or - * still cached simply return it from the cache, otherwise fetch it directly from zookeeper and - * retain the value for at least cacheForMillis milliseconds. Cached properties are watched in - * zookeeper and updated automatically. This version of {@code getCollectionProperties} should be - * used when properties need to be consulted frequently in the absence of an active {@link - * CollectionPropsWatcher}. - * - * @param collection The collection for which properties are desired - * @param cacheForMillis The minimum number of milliseconds to maintain a cache for the specified - * collection's properties. Setting a {@code CollectionPropsWatcher} will override this value - * and retain the cache for the life of the watcher. A lack of changes in zookeeper may allow - * the caching to remain for a greater duration up to the cycle time of {@code CacheCleaner}. - * Passing zero for this value will explicitly remove the cached copy if and only if it is due - * to expire and no watch exists. Any positive value will extend the expiration time if - * required. - * @return a map representing the key/value properties for the collection. - */ + /** Get and cache collection properties for a given collection */ public Map getCollectionProperties(final String collection, long cacheForMillis) { - synchronized (watchedCollectionProps) { // making decisions based on the result of a get... - Watcher watcher = null; - if (cacheForMillis > 0) { - watcher = - collectionPropsWatchers.compute( - collection, - (c, w) -> - w == null ? new PropsWatcher(c, cacheForMillis) : w.renew(cacheForMillis)); - } - VersionedCollectionProps vprops = watchedCollectionProps.get(collection); - boolean haveUnexpiredProps = vprops != null && vprops.cacheUntilNs > System.nanoTime(); - long untilNs = - System.nanoTime() + TimeUnit.NANOSECONDS.convert(cacheForMillis, TimeUnit.MILLISECONDS); - Map properties; - if (haveUnexpiredProps) { - properties = vprops.props; - vprops.cacheUntilNs = Math.max(vprops.cacheUntilNs, untilNs); - } else { - try { - VersionedCollectionProps vcp = fetchCollectionProperties(collection, watcher); - properties = vcp.props; - if (cacheForMillis > 0) { - vcp.cacheUntilNs = untilNs; - watchedCollectionProps.put(collection, vcp); - } else { - // we're synchronized on watchedCollectionProps and we can only get here if we have - // found an expired vprops above, so it is safe to remove the cached value and let the - // GC free up some mem a bit sooner. - if (!collectionPropsObservers.containsKey(collection)) { - watchedCollectionProps.remove(collection); - } - } - } catch (Exception e) { - throw new SolrException( - ErrorCode.SERVER_ERROR, - "Error reading collection properties", - SolrZkClient.checkInterrupted(e)); - } - } - return properties; - } - } - - private static class VersionedCollectionProps { - int zkVersion; - Map props; - long cacheUntilNs = 0; - - VersionedCollectionProps(int zkVersion, Map props) { - this.zkVersion = zkVersion; - this.props = props; - } + return collectionPropertiesZkStateReader.getCollectionProperties(collection, cacheForMillis); } static String getCollectionPropsPath(final String collection) { - return COLLECTIONS_ZKNODE + '/' + collection + '/' + COLLECTION_PROPS_ZKNODE; - } - - private VersionedCollectionProps fetchCollectionProperties(String collection, Watcher watcher) - throws KeeperException, InterruptedException { - final String znodePath = getCollectionPropsPath(collection); - // lazy init cache cleaner once we know someone is using collection properties. - if (collectionPropsCacheCleaner == null) { - synchronized (this) { // There can be only one! :) - if (collectionPropsCacheCleaner == null) { - collectionPropsCacheCleaner = notifications.submit(new CacheCleaner()); - } - } - } - while (true) { - try { - Stat stat = new Stat(); - byte[] data = zkClient.getData(znodePath, watcher, stat, true); - @SuppressWarnings("unchecked") - Map props = (Map) Utils.fromJSON(data); - return new VersionedCollectionProps(stat.getVersion(), props); - } catch (ClassCastException e) { - throw new SolrException( - ErrorCode.SERVER_ERROR, - "Unable to parse collection properties for collection " + collection, - e); - } catch (KeeperException.NoNodeException e) { - if (watcher != null) { - // Leave an exists watch in place in case a collectionprops.json is created later. - Stat exists = zkClient.exists(znodePath, watcher, true); - if (exists != null) { - // Rare race condition, we tried to fetch the data and couldn't find it, then we found - // it exists. Loop and try again. - continue; - } - } - return new VersionedCollectionProps(-1, emptyMap()); - } - } + return CollectionPropertiesZkStateReader.getCollectionPropsPath(collection); } /** @@ -1453,101 +1319,6 @@ private void refreshAndWatchChildren() throws KeeperException, InterruptedExcept } } - /** Watches collection properties */ - class PropsWatcher implements Watcher { - private final String coll; - private long watchUntilNs; - - PropsWatcher(String coll) { - this.coll = coll; - watchUntilNs = 0; - } - - PropsWatcher(String coll, long forMillis) { - this.coll = coll; - watchUntilNs = - System.nanoTime() + TimeUnit.NANOSECONDS.convert(forMillis, TimeUnit.MILLISECONDS); - } - - public PropsWatcher renew(long forMillis) { - watchUntilNs = - System.nanoTime() + TimeUnit.NANOSECONDS.convert(forMillis, TimeUnit.MILLISECONDS); - return this; - } - - @Override - public void process(WatchedEvent event) { - // session events are not change events, and do not remove the watcher - if (EventType.None.equals(event.getType())) { - return; - } - - boolean expired = System.nanoTime() > watchUntilNs; - if (!collectionPropsObservers.containsKey(coll) && expired) { - // No one can be notified of the change, we can ignore it and "unset" the watch - log.debug("Ignoring property change for collection {}", coll); - return; - } - - log.info( - "A collection property change: [{}] for collection [{}] has occurred - updating...", - event, - coll); - - refreshAndWatch(true); - } - - /** - * Refresh collection properties from ZK and leave a watch for future changes. Updates the - * properties in watchedCollectionProps with the results of the refresh. Optionally notifies - * watchers - */ - void refreshAndWatch(boolean notifyWatchers) { - try { - synchronized (watchedCollectionProps) { // making decisions based on the result of a get... - VersionedCollectionProps vcp = fetchCollectionProperties(coll, this); - Map properties = vcp.props; - VersionedCollectionProps existingVcp = watchedCollectionProps.get(coll); - if (existingVcp == null - || // never called before, record what we found - vcp.zkVersion > existingVcp.zkVersion - || // newer info we should update - vcp.zkVersion == -1) { // node was deleted start over - watchedCollectionProps.put(coll, vcp); - if (notifyWatchers) { - notifyPropsWatchers(coll, properties); - } - if (vcp.zkVersion == -1 && existingVcp != null) { // Collection DELETE detected - - // We should not be caching a collection that has been deleted. - watchedCollectionProps.remove(coll); - - // core ref counting not relevant here, don't need canRemove(), we just sent - // a notification of an empty set of properties, no reason to watch what doesn't - // exist. - collectionPropsObservers.remove(coll); - - // This is the one time we know it's safe to throw this out. We just failed to set the - // watch due to an NoNodeException, so it isn't held by ZK and can't re-set itself due - // to an update. - collectionPropsWatchers.remove(coll); - } - } - } - } catch (KeeperException.SessionExpiredException - | KeeperException.ConnectionLossException e) { - log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: ", e); - } catch (KeeperException e) { - log.error("Lost collection property watcher for {} due to ZK error", coll, e); - throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "A ZK error has occurred", e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - log.error( - "Lost collection property watcher for {} due to the thread being interrupted", coll, e); - } - } - } - /** Watches /collections children . */ class CollectionsChildWatcher implements Watcher { @@ -2069,38 +1840,12 @@ Map getCollectionWatches() { public void registerCollectionPropsWatcher( final String collection, CollectionPropsWatcher propsWatcher) { - AtomicBoolean watchSet = new AtomicBoolean(false); - collectionPropsObservers.compute( - collection, - (k, v) -> { - if (v == null) { - v = new CollectionWatch<>(); - watchSet.set(true); - } - v.stateWatchers.add(propsWatcher); - return v; - }); - if (watchSet.get()) { - collectionPropsWatchers.computeIfAbsent(collection, PropsWatcher::new).refreshAndWatch(false); - } + collectionPropertiesZkStateReader.registerCollectionPropsWatcher(collection, propsWatcher); } public void removeCollectionPropsWatcher(String collection, CollectionPropsWatcher watcher) { - collectionPropsObservers.compute( - collection, - (k, v) -> { - if (v == null) return null; - v.stateWatchers.remove(watcher); - if (v.canBeRemoved()) { - // don't want this to happen in middle of other blocks that might add it back. - synchronized (watchedCollectionProps) { - watchedCollectionProps.remove(collection); - } - return null; - } - return v; - }); + collectionPropertiesZkStateReader.removeCollectionPropsWatcher(collection, watcher); } public static class ConfigData { @@ -2352,66 +2097,6 @@ private boolean setIfNewer(SolrZkClient.NodeData n) { } } - private void notifyPropsWatchers(String collection, Map properties) { - try { - collectionPropsNotifications.submit(new PropsNotification(collection, properties)); - } catch (RejectedExecutionException e) { - if (!closed) { - log.error("Couldn't run collection properties notifications for {}", collection, e); - } - } - } - - private class PropsNotification implements Runnable { - - private final String collection; - private final Map collectionProperties; - private final List watchers = new ArrayList<>(); - - private PropsNotification(String collection, Map collectionProperties) { - this.collection = collection; - this.collectionProperties = collectionProperties; - // guarantee delivery of notification regardless of what happens to collectionPropsObservers - // while we wait our turn in the executor by capturing the list on creation. - collectionPropsObservers.compute( - collection, - (k, v) -> { - if (v == null) return null; - watchers.addAll(v.stateWatchers); - return v; - }); - } - - @Override - public void run() { - for (CollectionPropsWatcher watcher : watchers) { - if (watcher.onStateChanged(collectionProperties)) { - removeCollectionPropsWatcher(collection, watcher); - } - } - } - } - - private class CacheCleaner implements Runnable { - @Override - public void run() { - while (!Thread.interrupted()) { - try { - Thread.sleep(60000); - } catch (InterruptedException e) { - // Executor shutdown will send us an interrupt - break; - } - watchedCollectionProps - .entrySet() - .removeIf( - entry -> - entry.getValue().cacheUntilNs < System.nanoTime() - && !collectionPropsObservers.containsKey(entry.getKey())); - } - } - } - /** * Helper class that acts as both a {@link DocCollectionWatcher} and a {@link LiveNodesListener} * while wraping and delegating to a {@link CollectionStateWatcher} From b0ab58ef87538ac05ab73d9e2e4cdb6fba4aac56 Mon Sep 17 00:00:00 2001 From: Houston Putman Date: Mon, 29 Jul 2024 12:47:57 -0400 Subject: [PATCH 04/38] SOLR-12963: Default uninvertible=false in schema version 1.7 (#2601) --- solr/CHANGES.txt | 7 ++++ .../org/apache/solr/schema/FieldType.java | 2 +- .../apache/solr/schema/SortableTextField.java | 2 +- .../solr/uninverting/UninvertingReader.java | 2 +- .../solr/collection1/conf/schema-collate.xml | 12 +++---- .../collection1/conf/schema-custom-field.xml | 4 +-- .../conf/schema-distributed-missing-sort.xml | 36 +++++++++---------- .../collection1/conf/schema-docValues.xml | 10 +++--- .../conf/schema-docValuesFaceting.xml | 36 +++++++++---------- .../collection1/conf/schema-docValuesJoin.xml | 26 +++++++------- .../schema-id-and-version-fields-only.xml | 2 +- .../conf/schema-multiword-synonyms.xml | 4 +-- .../solr/collection1/conf/schema-numeric.xml | 30 ++++++++-------- .../collection1/conf/schema-pseudo-fields.xml | 4 +-- .../solr/collection1/conf/schema-rest.xml | 22 ++++++------ .../collection1/conf/schema-sorting-text.xml | 2 +- .../solr/collection1/conf/schema-spatial.xml | 8 ++--- .../solr/collection1/conf/schema_latest.xml | 20 +++++------ .../configsets/cloud-dynamic/conf/schema.xml | 22 ++++++------ .../cloud-managed/conf/managed-schema.xml | 6 ++-- .../configsets/cloud-minimal/conf/schema.xml | 4 +-- .../solr/configsets/dedup/conf/schema.xml | 2 +- .../collectionA/conf/schema.xml | 2 +- .../collectionB/conf/schema.xml | 2 +- .../configsets/doc-expiry/conf/schema.xml | 2 +- .../exitable-directory/conf/schema.xml | 6 ++-- .../designer/TestSchemaDesignerAPI.java | 14 ++++++-- .../solr/rest/schema/TestFieldResource.java | 4 +-- .../SchemaVersionSpecificBehaviorTest.java | 8 +++-- .../solr/collection1/conf/schema.xml | 14 ++++---- .../solr/configsets/sql/conf/schema.xml | 2 +- .../solr/collection1/conf/managed-schema.xml | 6 ++-- .../_default/conf/managed-schema.xml | 2 +- .../conf/managed-schema.xml | 8 ++--- ...field-type-definitions-and-properties.adoc | 4 +-- .../pages/major-changes-in-solr-9.adoc | 3 ++ .../solrj/solr/configsets/ml/conf/schema.xml | 16 ++++----- .../solr/configsets/streaming/conf/schema.xml | 10 +++--- .../collection1/conf/schema-replication1.xml | 2 +- .../solr/collection1/conf/schema-sql.xml | 10 +++--- .../solrj/solr/collection1/conf/schema.xml | 18 +++++----- .../solrj/solr/configsets/ml/conf/schema.xml | 12 +++---- .../solr/configsets/shared/conf/schema.xml | 2 +- .../solr/configsets/streaming/conf/schema.xml | 10 +++--- .../tracking-updates/conf/schema.xml | 4 +-- .../solr/multicore/core0/conf/schema.xml | 2 +- .../solr/multicore/core1/conf/schema.xml | 2 +- .../solr/collection1/conf/schema.xml | 26 +++++++------- 48 files changed, 238 insertions(+), 216 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 5676087acec..883ee4cd5d5 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -156,6 +156,13 @@ Improvements Statuses are now removed 5 minutes after the read of a completed/failed status. Helps collection async backup/restore and other operations scale to 100+ shards. (Pierre Salagnac, David Smiley) +* SOLR-10808, SOLR-12963: The Solr schema version has been increased to 1.7. + Starting in schema version 1.7, all fields/fieldTypes that support docValues will have them enabled by default. + This behavior can be reverted by setting the 'docValues' parameter for a field or a field type to false, the default for schema versions 1.6 and below. + Also in schema version 1.7, all fields/fieldTypes will be unable to be uninverted by default. + This behavior can be reverted by setting the 'uninvertible' parameter for a field or a field type to true, the default for schema versions 1.6 and below. + (Houston Putman, hossman) + * SOLR-10808 : The Solr schema version has been increased to 1.7. Since schema version 1.7, all fields/fieldTypes that support docValues will have them enabled by default. This behavior can be reverted by setting 'docValues' parameter for a field or a field type to false, the default for schema versions 1.6 and below. (Houston Putman) diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java index 9eae31f1a02..9431691b1de 100644 --- a/solr/core/src/java/org/apache/solr/schema/FieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java @@ -190,7 +190,7 @@ protected void setArgs(IndexSchema schema, Map args) { if (schemaVersion >= 1.6f) properties |= USE_DOCVALUES_AS_STORED; if (schemaVersion >= 1.7f && doesTypeSupportDocValues()) properties |= DOC_VALUES; - properties |= UNINVERTIBLE; + if (schemaVersion < 1.7f) properties |= UNINVERTIBLE; this.args = Collections.unmodifiableMap(args); Map initArgs = new HashMap<>(args); diff --git a/solr/core/src/java/org/apache/solr/schema/SortableTextField.java b/solr/core/src/java/org/apache/solr/schema/SortableTextField.java index b61d668f333..4811a1dfa59 100644 --- a/solr/core/src/java/org/apache/solr/schema/SortableTextField.java +++ b/solr/core/src/java/org/apache/solr/schema/SortableTextField.java @@ -227,7 +227,7 @@ public ValueSource getSingleValueSource( return new SortedSetFieldSource(field.getName(), selectorType); } - /** {@inheritDoc} this field type is not uninvertable, this method always returns null */ + /** {@inheritDoc} this field type is not uninvertible, this method always returns null */ @Override public Type getUninversionType(SchemaField sf) { return null; diff --git a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java index 78a3576420d..174095a2ea4 100644 --- a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java +++ b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java @@ -228,7 +228,7 @@ public static LeafReader wrap(LeafReader in, Function mapping) { ArrayList newFieldInfos = new ArrayList<>(in.getFieldInfos().size()); for (FieldInfo fi : in.getFieldInfos()) { DocValuesType type = fi.getDocValuesType(); - // fields which currently don't have docValues, but are uninvertable (indexed or points data + // fields which currently don't have docValues, but are uninvertible (indexed or points data // present) if (type == DocValuesType.NONE && (fi.getIndexOptions() != IndexOptions.NONE diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml b/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml index c9587d75f7d..4a4dbf42a01 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-collate.xml @@ -39,12 +39,12 @@ - - - - - - + + + + + + id diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml b/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml index db038b62787..1b36f23d497 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-custom-field.xml @@ -17,8 +17,8 @@ --> - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml b/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml index 84d628f7da5..90caa68a3c8 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-distributed-missing-sort.xml @@ -17,30 +17,30 @@ --> - - + + - - + + - - + + - - + + - - + + - - + + + sortMissingLast="true" uninvertible="true"/> + sortMissingFirst="true" uninvertible="true"/> @@ -70,12 +70,12 @@ - + - - + + - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml index ddb42f36e8f..74f377c6640 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml @@ -38,14 +38,14 @@ These are provided more for backward compatability, allowing one to create a schema that matches an existing lucene index. --> - - - - + + + + - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml index 8db18dc2996..67e6789f610 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml @@ -17,11 +17,11 @@ --> - - - - - + + + + + @@ -30,41 +30,41 @@ - + - + - + - + - + - + - + - + - + - + - + - + - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml index 646956e43a6..0fccb86224f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml @@ -28,7 +28,7 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> @@ -65,33 +65,33 @@ - + - + - + - + - + - + - - + + - - + + - + - + id diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml b/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml index a98b16a6b84..a88da298762 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-id-and-version-fields-only.xml @@ -17,7 +17,7 @@ --> - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml b/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml index 92989ce2b64..f6cfe7f8b94 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml @@ -19,8 +19,8 @@ - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml b/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml index 78b13270b80..3a487dfea02 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-numeric.xml @@ -29,37 +29,37 @@ + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> + positionIncrementGap="0" uninvertible="true"/> diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-pseudo-fields.xml b/solr/core/src/test-files/solr/collection1/conf/schema-pseudo-fields.xml index 19bedf66555..a5d87f03c86 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-pseudo-fields.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-pseudo-fields.xml @@ -37,8 +37,8 @@ id - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml index 2f5e04790fd..517182771bd 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml @@ -30,17 +30,17 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> - - - - + + + + - - - - + + + + @@ -97,8 +97,8 @@ - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-sorting-text.xml b/solr/core/src/test-files/solr/collection1/conf/schema-sorting-text.xml index cc814ab58ca..fdc675e2d07 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-sorting-text.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-sorting-text.xml @@ -46,7 +46,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml index bfee166fb0d..7ec98bde0f7 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml @@ -18,12 +18,12 @@ - - - + + + - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml index 100abb67981..7d1e09faf28 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml @@ -33,7 +33,7 @@ 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> - - - - - + + + + + - - - - + + + + - - - - + + + + @@ -96,8 +96,8 @@ - - + + diff --git a/solr/core/src/test-files/solr/configsets/cloud-managed/conf/managed-schema.xml b/solr/core/src/test-files/solr/configsets/cloud-managed/conf/managed-schema.xml index 2714a21b5de..d457870208a 100644 --- a/solr/core/src/test-files/solr/configsets/cloud-managed/conf/managed-schema.xml +++ b/solr/core/src/test-files/solr/configsets/cloud-managed/conf/managed-schema.xml @@ -17,12 +17,12 @@ --> - - + + - + diff --git a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/schema.xml index b539459ad5e..fc23706512e 100644 --- a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/schema.xml @@ -17,8 +17,8 @@ --> - - + + diff --git a/solr/core/src/test-files/solr/configsets/dedup/conf/schema.xml b/solr/core/src/test-files/solr/configsets/dedup/conf/schema.xml index fadaa72eac9..2da9becb844 100644 --- a/solr/core/src/test-files/solr/configsets/dedup/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/dedup/conf/schema.xml @@ -18,7 +18,7 @@ - + diff --git a/solr/core/src/test-files/solr/configsets/different-stopwords/collectionA/conf/schema.xml b/solr/core/src/test-files/solr/configsets/different-stopwords/collectionA/conf/schema.xml index 1875c733f96..0d788eb038a 100644 --- a/solr/core/src/test-files/solr/configsets/different-stopwords/collectionA/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/different-stopwords/collectionA/conf/schema.xml @@ -30,7 +30,7 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> diff --git a/solr/core/src/test-files/solr/configsets/different-stopwords/collectionB/conf/schema.xml b/solr/core/src/test-files/solr/configsets/different-stopwords/collectionB/conf/schema.xml index 98fc0a8c5c9..37692c15943 100644 --- a/solr/core/src/test-files/solr/configsets/different-stopwords/collectionB/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/different-stopwords/collectionB/conf/schema.xml @@ -30,7 +30,7 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> diff --git a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml index 62d52896700..8a4eb25bba9 100644 --- a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml @@ -30,7 +30,7 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> diff --git a/solr/core/src/test-files/solr/configsets/exitable-directory/conf/schema.xml b/solr/core/src/test-files/solr/configsets/exitable-directory/conf/schema.xml index 4ade7650b81..e9090724541 100644 --- a/solr/core/src/test-files/solr/configsets/exitable-directory/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/exitable-directory/conf/schema.xml @@ -16,11 +16,11 @@ limitations under the License. --> - + - - + + diff --git a/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java index 90a7685d83e..f059acb8a6c 100644 --- a/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java +++ b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java @@ -839,6 +839,7 @@ public void testSchemaDiffEndpoint() throws Exception { SimpleOrderedMap idFieldMapUpdated = idFieldMap.clone(); idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("docValues", 0), Boolean.FALSE); idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("useDocValuesAsStored", 0), Boolean.FALSE); + idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("uninvertible", 0), Boolean.TRUE); idFieldMapUpdated.setVal( idFieldMapUpdated.indexOf("omitTermFreqAndPositions", 0), Boolean.FALSE); @@ -892,14 +893,23 @@ public void testSchemaDiffEndpoint() throws Exception { assertEquals( Arrays.asList( Map.of( - "omitTermFreqAndPositions", true, "useDocValuesAsStored", true, "docValues", true), + "omitTermFreqAndPositions", + true, + "useDocValuesAsStored", + true, + "docValues", + true, + "uninvertible", + false), Map.of( "omitTermFreqAndPositions", false, "useDocValuesAsStored", false, "docValues", - false)), + false, + "uninvertible", + true)), mapDiff.get("id")); assertNotNull(fieldsDiff.get("added")); Map fieldsAdded = (Map) fieldsDiff.get("added"); diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java index f019e73aef2..5eb97247a39 100644 --- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java +++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java @@ -30,7 +30,7 @@ public void testGetField() { "/response/lst[@name='field']/str[@name='type'] = 'text'", "/response/lst[@name='field']/bool[@name='indexed'] = 'true'", "/response/lst[@name='field']/bool[@name='stored'] = 'true'", - "/response/lst[@name='field']/bool[@name='uninvertible'] = 'true'", + "/response/lst[@name='field']/bool[@name='uninvertible'] = 'false'", "/response/lst[@name='field']/bool[@name='docValues'] = 'false'", "/response/lst[@name='field']/bool[@name='termVectors'] = 'true'", "/response/lst[@name='field']/bool[@name='termPositions'] = 'true'", @@ -64,7 +64,7 @@ public void testJsonGetField() throws Exception { "/field/type=='text'", "/field/indexed==true", "/field/stored==true", - "/field/uninvertible==true", + "/field/uninvertible==false", "/field/docValues==false", "/field/termVectors==true", "/field/termPositions==true", diff --git a/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java b/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java index 0ddbc7a28a6..41d64fb3cb6 100644 --- a/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java +++ b/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java @@ -76,9 +76,11 @@ public void testVersionBehavior() throws Exception { ((v < 1.7F || f.contains("text")) ? false : true), field.hasDocValues()); - // uninvertable defaults to true (for now) - assertTrue( - f + " field's type has wrong uninvertable for ver=" + ver, field.isUninvertible()); + // 1.7: uninvertible defaults to false + assertEquals( + f + " field's type has wrong uninvertible for ver=" + ver, + (v < 1.7F ? true : false), + field.isUninvertible()); } // regardless of version, explicit multiValued values on field or type diff --git a/solr/modules/ltr/src/test-files/solr/collection1/conf/schema.xml b/solr/modules/ltr/src/test-files/solr/collection1/conf/schema.xml index 07779ce8a93..56c756f75b5 100644 --- a/solr/modules/ltr/src/test-files/solr/collection1/conf/schema.xml +++ b/solr/modules/ltr/src/test-files/solr/collection1/conf/schema.xml @@ -22,8 +22,8 @@ - - + + @@ -73,11 +73,11 @@ - - - - - + + + + + diff --git a/solr/modules/sql/src/test-files/solr/configsets/sql/conf/schema.xml b/solr/modules/sql/src/test-files/solr/configsets/sql/conf/schema.xml index 7c7131ba66b..2311e766632 100644 --- a/solr/modules/sql/src/test-files/solr/configsets/sql/conf/schema.xml +++ b/solr/modules/sql/src/test-files/solr/configsets/sql/conf/schema.xml @@ -43,7 +43,7 @@ 1.4: default auto-phrase (QueryParser feature) to off 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> diff --git a/solr/prometheus-exporter/src/test-files/solr/collection1/conf/managed-schema.xml b/solr/prometheus-exporter/src/test-files/solr/collection1/conf/managed-schema.xml index 76a6f64af34..2c78c1667f0 100644 --- a/solr/prometheus-exporter/src/test-files/solr/collection1/conf/managed-schema.xml +++ b/solr/prometheus-exporter/src/test-files/solr/collection1/conf/managed-schema.xml @@ -55,7 +55,7 @@ 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> - + - + diff --git a/solr/server/solr/configsets/_default/conf/managed-schema.xml b/solr/server/solr/configsets/_default/conf/managed-schema.xml index 5250e837ff5..8d7b974bec0 100644 --- a/solr/server/solr/configsets/_default/conf/managed-schema.xml +++ b/solr/server/solr/configsets/_default/conf/managed-schema.xml @@ -55,7 +55,7 @@ 1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...) 1.6: useDocValuesAsStored defaults to true. - 1.7: docValues defaults to true. + 1.7: docValues defaults to true, uninvertible defaults to false. --> @@ -134,11 +134,11 @@ - + - - + + diff --git a/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc b/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc index 6122e7476d2..f085682b6cb 100644 --- a/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc +++ b/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc @@ -198,10 +198,10 @@ The table below includes the default value for most `FieldType` implementations |Property |Description |Implicit Default |`indexed` |If `true`, the value of the field can be used in queries to retrieve matching documents. |`true` |`stored` |If `true`, the actual value of the field can be retrieved by queries. |`true` -|`docValues` |If `true`, the value of the field will be put in a column-oriented xref:docvalues.adoc[] structure. |`false` +|`docValues` |If `true`, the value of the field will be put in a column-oriented xref:docvalues.adoc[] structure. |`true` |`sortMissingFirst`, `sortMissingLast` |Control the placement of documents when a sort field is not present. |`false` |`multiValued` |If `true`, indicates that a single document might contain multiple values for this field type. |`false` -|`uninvertible` |If `true`, indicates that an `indexed="true" docValues="false"` field can be "un-inverted" at query time to build up large in memory data structure to serve in place of xref:docvalues.adoc[]. *Defaults to `true` for historical reasons, but users are strongly encouraged to set this to `false` for stability and use `docValues="true"` as needed.* |`true` +|`uninvertible` |If `true`, indicates that an `indexed="true" docValues="false"` field can be "un-inverted" at query time to build up large in memory data structure to serve in place of xref:docvalues.adoc[]. |`false` |`omitNorms` |If `true`, omits the norms associated with this field (this disables length normalization for the field, and saves some memory). *Defaults to true for all primitive (non-analyzed) field types, such as int, float, data, bool, and string.* Only full-text fields or fields that need an index-time boost need norms. |* |`omitTermFreqAndPositions` |If `true`, omits term frequency, positions, and payloads from postings for this field. This can be a performance boost for fields that don't require that information. It also reduces the storage space required for the index. Queries that rely on position that are issued on a field with this option will silently fail to find documents. *This property defaults to true for all field types that are not text fields.* |* |`omitPositions` |Similar to `omitTermFreqAndPositions` but preserves term frequency information. |* diff --git a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc index 6f9123bded5..ee1cae69deb 100644 --- a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc +++ b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc @@ -72,6 +72,7 @@ Due to changes in Lucene 9, that isn't possible any more. The default schemaVersion is now 1.7. With the new schema version, all fields and fieldTypes that support docValues (StrField, *PointField, BoolField, etc.) will have docValues enabled by default. +Also, fields will be unable to be uninverted by default. (`uninvertible=false`) In order to upgrade from schemaVersion 1.6 to 1.7, a re-index of all data will generally be required. **This re-index cannot be in-place**. @@ -82,6 +83,8 @@ However, if any of the following is true, a re-index is not required: * All applicable fields already have docValues enabled in the original schema * The schema is updated to explicitly default `docValues="false"` for all fields and fieldTypes that did not have an explicit `docValues` default provided. +The `uninvertible=false` default has no impact on the index, so is unrelated to re-indexing concerns. + == Solr 9.6 === Dependency upgrades Lucene is upgraded to 9.10.0 and a variety of library dependencies have been upgraded. See https://solr.apache.org/docs/9_6_0/changes/Changes.html for specific upgraded libraries. diff --git a/solr/solrj-streaming/src/test-files/solrj/solr/configsets/ml/conf/schema.xml b/solr/solrj-streaming/src/test-files/solrj/solr/configsets/ml/conf/schema.xml index 74b073ae8b6..35099afce8e 100644 --- a/solr/solrj-streaming/src/test-files/solrj/solr/configsets/ml/conf/schema.xml +++ b/solr/solrj-streaming/src/test-files/solrj/solr/configsets/ml/conf/schema.xml @@ -29,13 +29,13 @@ - - + + - - - - + + + + @@ -45,8 +45,8 @@ - - + + diff --git a/solr/solrj-streaming/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml b/solr/solrj-streaming/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml index 5784fd0ffe4..5a202baa2b8 100644 --- a/solr/solrj-streaming/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml +++ b/solr/solrj-streaming/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml @@ -43,12 +43,12 @@ - + - - - - + + + + diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-replication1.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-replication1.xml index 691ada4ba64..e0d04c62517 100644 --- a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-replication1.xml +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-replication1.xml @@ -27,7 +27,7 @@ - + diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml index 08dd32abf9d..f557a80e64d 100644 --- a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml @@ -45,12 +45,12 @@ - + - - - - + + + + - - - - - - - - - + + + + + + + + + diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/ml/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/ml/conf/schema.xml index 544edf8ae04..be212b86919 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/ml/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/ml/conf/schema.xml @@ -29,13 +29,13 @@ - - + + - - - - + + + + diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/schema.xml index 4aad5690101..f12c5e09038 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/schema.xml @@ -42,7 +42,7 @@ - + diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml index 5784fd0ffe4..5a202baa2b8 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml @@ -43,12 +43,12 @@ - + - - - - + + + + diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml index b539459ad5e..fc23706512e 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml @@ -17,8 +17,8 @@ --> - - + + diff --git a/solr/solrj/src/test-files/solrj/solr/multicore/core0/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/multicore/core0/conf/schema.xml index 8b3a3933996..8dddc57a24e 100644 --- a/solr/solrj/src/test-files/solrj/solr/multicore/core0/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/multicore/core0/conf/schema.xml @@ -19,7 +19,7 @@ - + diff --git a/solr/solrj/src/test-files/solrj/solr/multicore/core1/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/multicore/core1/conf/schema.xml index fd1cac0374f..079a2683edb 100644 --- a/solr/solrj/src/test-files/solrj/solr/multicore/core1/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/multicore/core1/conf/schema.xml @@ -19,7 +19,7 @@ - + diff --git a/solr/test-framework/src/test-files/solr/collection1/conf/schema.xml b/solr/test-framework/src/test-files/solr/collection1/conf/schema.xml index f18a9b7490f..e65df97ed61 100644 --- a/solr/test-framework/src/test-files/solr/collection1/conf/schema.xml +++ b/solr/test-framework/src/test-files/solr/collection1/conf/schema.xml @@ -33,16 +33,16 @@ behavior of the fieldType. --> - - - - + + + + - - - - + + + + @@ -112,8 +112,8 @@ - - + + @@ -545,13 +545,13 @@ - + - + @@ -791,7 +791,7 @@ - + Date: Mon, 29 Jul 2024 17:22:26 -0500 Subject: [PATCH 05/38] SOLR-12963: Fix EnumFieldTest.testEnumSort --- .../test-files/solr/collection1/conf/schema-enums.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml b/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml index 40a6b4441ae..c9067dff946 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-enums.xml @@ -20,13 +20,13 @@ - + - - + + - + id @@ -44,6 +44,6 @@ but you can always add new values to the end. --> - + From adc3f7abbd3074dd974dd5a04705565ca69e5332 Mon Sep 17 00:00:00 2001 From: Gus Heck <46900717+gus-asf@users.noreply.github.com> Date: Tue, 30 Jul 2024 08:55:07 -0400 Subject: [PATCH 06/38] SOLR-17298 - ThreadCpuTimer safe for multi-threaded search (#2595) * SOLR-17298 Simplify ThreadCpuTimer by never instantiating it, and instead tracking any number of contexts per thread. This avoids thread safety issues previously caused by keeping state in a field. Ensure explicit correspondence between reported cpu time and the measurement of cpu used by limits when limits are in play. Also add TestInjection for cpu delay and use it in TestCpuAllowedLimit --- .../org/apache/solr/core/CoreContainer.java | 5 +- .../solr/handler/RequestHandlerBase.java | 49 ++++++---- .../apache/solr/request/SolrRequestInfo.java | 23 +---- .../apache/solr/search/CpuAllowedLimit.java | 66 ++++++++++--- .../solr/search/MultiThreadedSearcher.java | 5 +- .../org/apache/solr/search/QueryLimit.java | 31 ++++++ .../org/apache/solr/search/QueryLimits.java | 12 ++- .../apache/solr/search/SolrIndexSearcher.java | 12 +-- .../apache/solr/search/TimeAllowedLimit.java | 16 +++- .../org/apache/solr/util/TestInjection.java | 76 ++++++++++++++- .../org/apache/solr/util/ThreadCpuTimer.java | 96 +++++++++++-------- .../solr/search/CallerSpecificQueryLimit.java | 8 +- .../solr/search/TestCpuAllowedLimit.java | 62 +++++++++--- .../apache/solr/common/util/ExecutorUtil.java | 66 ++++++++++++- 14 files changed, 397 insertions(+), 130 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/search/QueryLimit.java diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index 174deadd4fb..ed0b2a48654 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -26,6 +26,7 @@ import static org.apache.solr.common.params.CommonParams.METRICS_PATH; import static org.apache.solr.common.params.CommonParams.ZK_PATH; import static org.apache.solr.common.params.CommonParams.ZK_STATUS_PATH; +import static org.apache.solr.search.CpuAllowedLimit.TIMING_CONTEXT; import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP; import com.github.benmanes.caffeine.cache.Interner; @@ -155,6 +156,7 @@ import org.apache.solr.util.OrderedExecutor; import org.apache.solr.util.RefCounted; import org.apache.solr.util.StartupLoggingUtils; +import org.apache.solr.util.ThreadCpuTimer; import org.apache.solr.util.stats.MetricUtils; import org.apache.zookeeper.KeeperException; import org.glassfish.hk2.utilities.binding.AbstractBinder; @@ -448,7 +450,8 @@ public CoreContainer(NodeConfig config, CoresLocator locator, boolean asyncSolrC ExecutorUtil.newMDCAwareFixedThreadPool( indexSearcherExecutorThreads, // thread count indexSearcherExecutorThreads * 1000, // queue size - new SolrNamedThreadFactory("searcherCollector")); + new SolrNamedThreadFactory("searcherCollector"), + () -> ThreadCpuTimer.reset(TIMING_CONTEXT)); } else { this.collectorExecutor = null; } diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java index ff020ddebd3..d85b939fbb5 100644 --- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java +++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java @@ -43,8 +43,9 @@ import org.apache.solr.metrics.SolrMetricsContext; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestHandler; -import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.search.CpuAllowedLimit; +import org.apache.solr.search.QueryLimits; import org.apache.solr.search.SyntaxError; import org.apache.solr.security.PermissionNameProvider; import org.apache.solr.update.processor.DistributedUpdateProcessor; @@ -62,6 +63,7 @@ public abstract class RequestHandlerBase ApiSupport, PermissionNameProvider { + public static final String REQUEST_CPU_TIMER_CONTEXT = "publishCpuTime"; protected NamedList initArgs = null; protected SolrParams defaults; protected SolrParams appends; @@ -217,12 +219,8 @@ public abstract void handleRequestBody(SolrQueryRequest req, SolrQueryResponse r @Override public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) { - ThreadCpuTimer threadCpuTimer = null; if (publishCpuTime) { - threadCpuTimer = - SolrRequestInfo.getRequestInfo() == null - ? new ThreadCpuTimer() - : SolrRequestInfo.getRequestInfo().getThreadCpuTimer(); + ThreadCpuTimer.beginContext(REQUEST_CPU_TIMER_CONTEXT); } HandlerMetrics metrics = getMetricsForThisRequest(req); metrics.requests.inc(); @@ -246,21 +244,36 @@ public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) { processErrorMetricsOnException(normalized, metrics); rsp.setException(normalized); } finally { - long elapsed = timer.stop(); - metrics.totalTime.inc(elapsed); - - if (publishCpuTime && threadCpuTimer != null) { - Optional cpuTime = threadCpuTimer.getElapsedCpuMs(); - if (cpuTime.isPresent()) { - // add CPU_TIME if not already added by SearchHandler - NamedList header = rsp.getResponseHeader(); - if (header != null) { - if (header.get(ThreadCpuTimer.CPU_TIME) == null) { - header.add(ThreadCpuTimer.CPU_TIME, cpuTime.get()); + try { + long elapsed = timer.stop(); + metrics.totalTime.inc(elapsed); + + if (publishCpuTime) { + Optional cpuTime = ThreadCpuTimer.readMSandReset(REQUEST_CPU_TIMER_CONTEXT); + if (QueryLimits.getCurrentLimits().isLimitsEnabled()) { + // prefer the value from the limit if available to avoid confusing users with trivial + // differences. Not fond of the spotless formatting here... + cpuTime = + Optional.ofNullable( + (Long) + QueryLimits.getCurrentLimits() + .currentLimitValueFor(CpuAllowedLimit.class) + .orElse(cpuTime.orElse(null))); + } + if (cpuTime.isPresent()) { + // add CPU_TIME if not already added by SearchHandler + NamedList header = rsp.getResponseHeader(); + if (header != null) { + if (header.get(ThreadCpuTimer.CPU_TIME) == null) { + header.add(ThreadCpuTimer.CPU_TIME, cpuTime.get()); + } } + rsp.addToLog(ThreadCpuTimer.LOCAL_CPU_TIME, cpuTime.get()); } - rsp.addToLog(ThreadCpuTimer.LOCAL_CPU_TIME, cpuTime.get()); } + } finally { + // whatever happens be sure to clear things out at end of request. + ThreadCpuTimer.reset(); } } } diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java index b7d0a11eb97..193b838e9cd 100644 --- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java +++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java @@ -33,7 +33,6 @@ import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.search.QueryLimits; import org.apache.solr.servlet.SolrDispatchFilter; -import org.apache.solr.util.ThreadCpuTimer; import org.apache.solr.util.TimeZoneUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +45,6 @@ public class SolrRequestInfo { private static final ThreadLocal> threadLocal = ThreadLocal.withInitial(ArrayDeque::new); static final Object LIMITS_KEY = new Object(); - static final Object CPU_TIMER_KEY = new Object(); private int refCount = 1; // prevent closing when still used @@ -80,10 +78,12 @@ public static void setRequestInfo(SolrRequestInfo info) { assert false : "SolrRequestInfo Stack is full"; log.error("SolrRequestInfo Stack is full"); } else if (!stack.isEmpty() && info.req != null) { - // New SRI instances inherit limits and thread CPU from prior SRI regardless of parameters. + // New SRI instances inherit limits from prior SRI regardless of parameters. // This ensures these two properties cannot be changed or removed for a given thread once set. // if req is null then limits will be an empty instance with no limits anyway. - info.req.getContext().put(CPU_TIMER_KEY, stack.peek().getThreadCpuTimer()); + + // protected by !stack.isEmpty() + // noinspection DataFlowIssue info.req.getContext().put(LIMITS_KEY, stack.peek().getLimits()); } // this creates both new QueryLimits and new ThreadCpuTime if not already set @@ -244,27 +244,12 @@ private void initQueryLimits() { */ public QueryLimits getLimits() { // make sure the ThreadCpuTime is always initialized - getThreadCpuTimer(); return req == null || rsp == null ? QueryLimits.NONE : (QueryLimits) req.getContext().computeIfAbsent(LIMITS_KEY, (k) -> new QueryLimits(req, rsp)); } - /** - * Get the thread CPU time monitor for the current request. This will either trigger the creation - * of a new instance if it hasn't been yet created, or will retrieve the already existing instance - * from the "bottom" of the request stack. - * - * @return the {@link ThreadCpuTimer} object for the current request. - */ - public ThreadCpuTimer getThreadCpuTimer() { - return req == null - ? new ThreadCpuTimer() - : (ThreadCpuTimer) - req.getContext().computeIfAbsent(CPU_TIMER_KEY, k -> new ThreadCpuTimer()); - } - public SolrDispatchFilter.Action getAction() { return action; } diff --git a/solr/core/src/java/org/apache/solr/search/CpuAllowedLimit.java b/solr/core/src/java/org/apache/solr/search/CpuAllowedLimit.java index aa3a1fae7b2..72362159f0b 100644 --- a/solr/core/src/java/org/apache/solr/search/CpuAllowedLimit.java +++ b/solr/core/src/java/org/apache/solr/search/CpuAllowedLimit.java @@ -16,14 +16,23 @@ */ package org.apache.solr.search; +import static org.apache.solr.util.ThreadCpuTimer.readNSAndReset; + import com.google.common.annotations.VisibleForTesting; +import java.lang.invoke.MethodHandles; +import java.text.DecimalFormat; +import java.text.DecimalFormatSymbols; +import java.util.Locale; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import net.jcip.annotations.NotThreadSafe; -import org.apache.lucene.index.QueryTimeout; import org.apache.solr.common.params.CommonParams; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; +import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.util.ThreadCpuTimer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Enforces a CPU-time based timeout on a given SolrQueryRequest, as specified by the {@code @@ -37,14 +46,18 @@ * @see ThreadCpuTimer */ @NotThreadSafe -public class CpuAllowedLimit implements QueryTimeout { - private final ThreadCpuTimer threadCpuTimer; +public class CpuAllowedLimit implements QueryLimit { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private final long requestedTimeoutNs; + private volatile long timedOutAt = 0L; + AtomicLong accumulatedTime = new AtomicLong(0); + public static final String TIMING_CONTEXT = CpuAllowedLimit.class.getName(); /** * Create an object to represent a CPU time limit for the current request. NOTE: this * implementation will attempt to obtain an existing thread CPU time monitor, created when {@link - * SolrRequestInfo#getThreadCpuTimer()} is initialized. + * QueryLimits#QueryLimits(SolrQueryRequest, SolrQueryResponse)} is called. * * @param req solr request with a {@code cpuAllowed} parameter */ @@ -52,11 +65,6 @@ public CpuAllowedLimit(SolrQueryRequest req) { if (!ThreadCpuTimer.isSupported()) { throw new IllegalArgumentException("Thread CPU time monitoring is not available."); } - SolrRequestInfo solrRequestInfo = SolrRequestInfo.getRequestInfo(); - // get existing timer if available to ensure sub-queries can't reset/exceed the intended time - // constraint. - threadCpuTimer = - solrRequestInfo != null ? solrRequestInfo.getThreadCpuTimer() : new ThreadCpuTimer(); long reqCpuLimit = req.getParams().getLong(CommonParams.CPU_ALLOWED, -1L); if (reqCpuLimit <= 0L) { @@ -65,11 +73,15 @@ public CpuAllowedLimit(SolrQueryRequest req) { } // calculate the time when the limit is reached, e.g. account for the time already spent requestedTimeoutNs = TimeUnit.NANOSECONDS.convert(reqCpuLimit, TimeUnit.MILLISECONDS); + + // here we rely on the current thread never creating a second CpuAllowedLimit within the same + // request, and also rely on it always creating a new CpuAllowedLimit object for each + // request that requires it. + ThreadCpuTimer.beginContext(TIMING_CONTEXT); } @VisibleForTesting CpuAllowedLimit(long limitMs) { - this.threadCpuTimer = new ThreadCpuTimer(); requestedTimeoutNs = TimeUnit.NANOSECONDS.convert(limitMs, TimeUnit.MILLISECONDS); } @@ -81,6 +93,38 @@ static boolean hasCpuLimit(SolrQueryRequest req) { /** Return true if usage has exceeded the limit. */ @Override public boolean shouldExit() { - return threadCpuTimer.getElapsedCpuNs() > requestedTimeoutNs; + if (timedOutAt > 0L) { + return true; + } + // if unsupported, use zero, and thus never exit, expect jvm and/or cpu branch + // prediction to short circuit things if unsupported. + Long delta = readNSAndReset(TIMING_CONTEXT).orElse(0L); + try { + if (accumulatedTime.addAndGet(delta) > requestedTimeoutNs) { + timedOutAt = accumulatedTime.get(); + return true; + } + return false; + } finally { + if (log.isTraceEnabled()) { + java.text.DecimalFormatSymbols symbols = new DecimalFormatSymbols(Locale.US); + DecimalFormat formatter = new DecimalFormat("#,###", symbols); + String threadName = Thread.currentThread().getName(); + String deltaFmt = formatter.format(delta); + String accumulated = formatter.format(accumulatedTime.get()); + String timeoutForComparison = formatter.format(requestedTimeoutNs); + log.trace( + "++++++++++++ SHOULD_EXIT - measuredDelta:{} accumulated:{} vs {} ++++ ON:{}", + deltaFmt, + accumulated, + timeoutForComparison, + threadName); + } + } + } + + @Override + public Object currentValue() { + return timedOutAt > 0 ? timedOutAt : accumulatedTime.get(); } } diff --git a/solr/core/src/java/org/apache/solr/search/MultiThreadedSearcher.java b/solr/core/src/java/org/apache/solr/search/MultiThreadedSearcher.java index f9810b1a083..b56931a650e 100644 --- a/solr/core/src/java/org/apache/solr/search/MultiThreadedSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/MultiThreadedSearcher.java @@ -120,10 +120,7 @@ SearchResult searchCollectorManagers( } static boolean allowMT(DelegatingCollector postFilter, QueryCommand cmd, Query query) { - if (postFilter != null - || cmd.getSegmentTerminateEarly() - || cmd.getTimeAllowed() > 0 - || !cmd.getMultiThreaded()) { + if (postFilter != null || cmd.getSegmentTerminateEarly() || !cmd.getMultiThreaded()) { return false; } else { MTCollectorQueryCheck allowMT = new MTCollectorQueryCheck(); diff --git a/solr/core/src/java/org/apache/solr/search/QueryLimit.java b/solr/core/src/java/org/apache/solr/search/QueryLimit.java new file mode 100644 index 00000000000..1043707f06b --- /dev/null +++ b/solr/core/src/java/org/apache/solr/search/QueryLimit.java @@ -0,0 +1,31 @@ +/* + * 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.search; + +import org.apache.lucene.index.QueryTimeout; + +public interface QueryLimit extends QueryTimeout { + /** + * A value representing the portion of the specified limit that has been consumed. Reading this + * value should never affect the outcome (other than the time it takes to do it). + * + * @return an expression of the amount of the limit used so far, numeric if possible, if + * non-numeric it should have toString() suitable for logging or similar expression to the + * user. + */ + Object currentValue(); +} diff --git a/solr/core/src/java/org/apache/solr/search/QueryLimits.java b/solr/core/src/java/org/apache/solr/search/QueryLimits.java index 2e232bccde6..86c7f488de3 100644 --- a/solr/core/src/java/org/apache/solr/search/QueryLimits.java +++ b/solr/core/src/java/org/apache/solr/search/QueryLimits.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Optional; import org.apache.lucene.index.QueryTimeout; import org.apache.solr.common.params.CommonParams; import org.apache.solr.request.SolrQueryRequest; @@ -34,7 +35,7 @@ * return true the next time it is checked (it may be checked in either Lucene code or Solr code) */ public class QueryLimits implements QueryTimeout { - private final List limits = + private final List limits = new ArrayList<>(3); // timeAllowed, cpu, and memory anticipated public static QueryLimits NONE = new QueryLimits(); @@ -149,6 +150,15 @@ public String limitStatusMessage() { return sb.toString(); } + public Optional currentLimitValueFor(Class limitClass) { + for (QueryLimit limit : limits) { + if (limit.getClass().isAssignableFrom(limitClass)) { + return Optional.of(limit.currentValue()); + } + } + return Optional.empty(); + } + /** Return true if there are any limits enabled for the current request. */ public boolean isLimitsEnabled() { return !limits.isEmpty(); diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index 8204487fed3..db3940f464e 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -1921,9 +1921,7 @@ public ScoreMode scoreMode() { final TopDocs topDocs; final ScoreMode scoreModeUsed; if (!MultiThreadedSearcher.allowMT(pf.postFilter, cmd, query)) { - if (log.isDebugEnabled()) { - log.debug("skipping collector manager"); - } + log.trace("SINGLE THREADED search, skipping collector manager in getDocListNC"); final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd); MaxScoreCollector maxScoreCollector = null; Collector collector = topCollector; @@ -1942,9 +1940,7 @@ public ScoreMode scoreMode() { ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f; } else { - if (log.isDebugEnabled()) { - log.debug("using CollectorManager"); - } + log.trace("MULTI-THREADED search, using CollectorManager int getDocListNC"); final MultiThreadedSearcher.SearchResult searchResult = new MultiThreadedSearcher(this) .searchCollectorManagers(len, cmd, query, true, needScores, false); @@ -2046,6 +2042,8 @@ public ScoreMode scoreMode() { } else { final TopDocs topDocs; if (!MultiThreadedSearcher.allowMT(pf.postFilter, cmd, query)) { + log.trace("SINGLE THREADED search, skipping collector manager in getDocListAndSetNC"); + @SuppressWarnings({"rawtypes"}) final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd); final DocSetCollector setCollector = new DocSetCollector(maxDoc); @@ -2072,7 +2070,7 @@ public ScoreMode scoreMode() { ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f; } else { - log.debug("using CollectorManager"); + log.trace("MULTI-THREADED search, using CollectorManager in getDocListAndSetNC"); boolean needMaxScore = needScores; MultiThreadedSearcher.SearchResult searchResult = diff --git a/solr/core/src/java/org/apache/solr/search/TimeAllowedLimit.java b/solr/core/src/java/org/apache/solr/search/TimeAllowedLimit.java index 432993d6c43..f837712176c 100644 --- a/solr/core/src/java/org/apache/solr/search/TimeAllowedLimit.java +++ b/solr/core/src/java/org/apache/solr/search/TimeAllowedLimit.java @@ -19,7 +19,6 @@ import static java.lang.System.nanoTime; import java.util.concurrent.TimeUnit; -import org.apache.lucene.index.QueryTimeout; import org.apache.solr.common.params.CommonParams; import org.apache.solr.request.SolrQueryRequest; @@ -30,9 +29,10 @@ * has {@code timeAllowed} set. Essentially only one timeAllowed can be specified for any thread * executing a query. This is to ensure that subqueries don't escape from the intended limit */ -public class TimeAllowedLimit implements QueryTimeout { +public class TimeAllowedLimit implements QueryLimit { private final long timeoutAt; + private final long timingSince; /** * Create an object to represent a time limit for the current request. @@ -50,9 +50,12 @@ public TimeAllowedLimit(SolrQueryRequest req) { throw new IllegalArgumentException( "Check for limit with hasTimeLimit(req) before creating a TimeAllowedLimit"); } - long timeAllowed = reqTimeAllowed - (long) req.getRequestTimer().getTime(); + long timeAlreadySpent = (long) req.getRequestTimer().getTime(); + long now = nanoTime(); + long timeAllowed = reqTimeAllowed - timeAlreadySpent; long nanosAllowed = TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS); - timeoutAt = nanoTime() + nanosAllowed; + timeoutAt = now + nanosAllowed; + timingSince = now - timeAlreadySpent; } /** Return true if the current request has a parameter with a valid value of the limit. */ @@ -65,4 +68,9 @@ static boolean hasTimeLimit(SolrQueryRequest req) { public boolean shouldExit() { return timeoutAt - nanoTime() < 0L; } + + @Override + public Object currentValue() { + return nanoTime() - timingSince; + } } diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java index 4f1547ccd00..8168f7beafe 100644 --- a/solr/core/src/java/org/apache/solr/util/TestInjection.java +++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java @@ -16,11 +16,18 @@ */ package org.apache.solr.util; +import com.google.common.util.concurrent.AtomicDouble; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.lang.reflect.Method; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.NoSuchAlgorithmException; +import java.text.DecimalFormat; +import java.text.DecimalFormatSymbols; import java.util.Collections; import java.util.HashSet; +import java.util.Locale; import java.util.Random; import java.util.Set; import java.util.Timer; @@ -32,13 +39,13 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.QueryTimeout; import org.apache.solr.common.NonExistentCoreException; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.util.Pair; import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; +import org.apache.solr.search.QueryLimit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,10 +161,26 @@ static Random random() { // non-private for testing public static volatile AtomicInteger countDocSetDelays = new AtomicInteger(0); - public static volatile QueryTimeout queryTimeout = null; + public static volatile QueryLimit queryTimeout = null; public static volatile boolean failInExecutePlanAction = false; + public static volatile AtomicInteger cpuTimerDelayInjectedNS = null; + + private static final KeyPairGenerator kpg; + + static { + KeyPairGenerator generator; + try { + generator = KeyPairGenerator.getInstance("RSA"); + } catch (NoSuchAlgorithmException e) { + generator = null; + } + kpg = generator; + } + + private static volatile AtomicDouble cpuLoadPerKey = null; + /** * Defaults to false, If set to true, then {@link * #injectSkipIndexWriterCommitOnClose} will return true @@ -531,6 +554,55 @@ public static boolean injectDirectUpdateLatch() { return true; } + public static void measureCpu() { + if (kpg == null || cpuLoadPerKey != null) { + return; + } + long start = System.nanoTime(); + for (int i = 0; i < 100; i++) { + genKeyPairAndDiscard(); + } + // note that this is potentially imprecise because our thread could get paused in the middle of + // this, but + // it should give us some notion + long end = System.nanoTime(); + cpuLoadPerKey = new AtomicDouble((end - start) / 100.0); + log.info("CPU per key = {}", cpuLoadPerKey); + } + + private static void genKeyPairAndDiscard() { + kpg.initialize(1024); + KeyPair kp = kpg.generateKeyPair(); + // avoid this getting optimized away by logging it + if (log.isTraceEnabled()) { + log.trace("{}", kp.getPublic()); + } + } + + private static void wasteCpu(int nanos) { + double wasteMe = nanos; + double loadPerKey = cpuLoadPerKey.get(); + if (loadPerKey > nanos) { + java.text.DecimalFormatSymbols symbols = new DecimalFormatSymbols(Locale.US); + + DecimalFormat formatter = new DecimalFormat("#,###.00", symbols); + // yes this is still wasting formatting when not warn, but not important here. + log.warn( + "Test requests smaller simulated cpu lag than a single keypair generation actual lag is {} ns", + formatter.format(loadPerKey)); + } + do { + genKeyPairAndDiscard(); + } while ((wasteMe = wasteMe - loadPerKey) > 0.0); + } + + public static void injectCpuUseInSearcherCpuLimitCheck() { + if (LUCENE_TEST_CASE == null) return; + if (cpuTimerDelayInjectedNS != null) { + wasteCpu(cpuTimerDelayInjectedNS.get()); + } + } + public static boolean injectReindexFailure() { if (reindexFailure != null) { Random rand = random(); diff --git a/solr/core/src/java/org/apache/solr/util/ThreadCpuTimer.java b/solr/core/src/java/org/apache/solr/util/ThreadCpuTimer.java index 054a5310d14..17d61fe3748 100644 --- a/solr/core/src/java/org/apache/solr/util/ThreadCpuTimer.java +++ b/solr/core/src/java/org/apache/solr/util/ThreadCpuTimer.java @@ -19,24 +19,19 @@ import java.lang.invoke.MethodHandles; import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; +import java.util.Map; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import net.jcip.annotations.NotThreadSafe; +import java.util.concurrent.atomic.AtomicLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Allows tracking information about the current thread using the JVM's built-in management bean - * {@link java.lang.management.ThreadMXBean}. - * - *

Calling code should create an instance of this class when starting the operation, and then can - * get the {@link #getElapsedCpuMs()} at any time thereafter. - * - *

This class is irrevocably not thread safe. Never allow instances of this class to be exposed - * to more than one thread. Acquiring an external lock will not be sufficient. This class can be - * considered "lock-hostile" due to its caching of timing information for a specific thread. + * {@link java.lang.management.ThreadMXBean}. Methods on this class are safe for use on any thread, + * but will return different values for different threads by design. */ -@NotThreadSafe public class ThreadCpuTimer { private static final long UNSUPPORTED = -1; public static final String CPU_TIME = "cpuTime"; @@ -59,14 +54,14 @@ public class ThreadCpuTimer { } } - private final long startCpuTimeNanos; + private static final ThreadLocal> threadLocalTimer = + ThreadLocal.withInitial(ConcurrentHashMap::new); - /** - * Create an instance to track the current thread's usage of CPU. Usage information can later be - * retrieved by calling {@link #getElapsedCpuMs()}. Timing starts immediately upon construction. - */ - public ThreadCpuTimer() { - this.startCpuTimeNanos = getThreadTotalCpuNs(); + /* no instances shall be created. */ + private ThreadCpuTimer() {} + + public static void beginContext(String context) { + readNSAndReset(context); } public static boolean isSupported() { @@ -74,47 +69,64 @@ public static boolean isSupported() { } /** - * Return CPU time consumed by this thread since the construction of this timer object. + * Get the number of nanoseconds since the last time this thread took a reading + * for the supplied context. * - * @return current value, or {@link #UNSUPPORTED} if not supported. + * @param context An arbitrary name that code can supply to avoid clashing with other usages. + * @return An optional long which may be empty if + * java.lang.management.ManagementFactory#getThreadMXBean() is unsupported or otherwise + * unavailable. */ - public long getElapsedCpuNs() { - return this.startCpuTimeNanos != UNSUPPORTED - ? getThreadTotalCpuNs() - this.startCpuTimeNanos - : UNSUPPORTED; + public static Optional readNSAndReset(String context) { + // simulate heavy query and/or heavy CPU load in tests + TestInjection.injectCpuUseInSearcherCpuLimitCheck(); + if (THREAD_MX_BEAN == null) { + return Optional.empty(); + } else { + AtomicLong threadCpuTime = + threadLocalTimer + .get() + .computeIfAbsent( + context, (ctx) -> new AtomicLong(THREAD_MX_BEAN.getCurrentThreadCpuTime())); + long currentThreadCpuTime = THREAD_MX_BEAN.getCurrentThreadCpuTime(); + long result = currentThreadCpuTime - threadCpuTime.get(); + threadCpuTime.set(currentThreadCpuTime); + return Optional.of(result); + } } /** - * Get the cpu time for the current thread since {@link Thread#start()} without throwing an - * exception. + * Discard any accumulated time for a given context since the last invocation. * - * @see ThreadMXBean#getCurrentThreadCpuTime() for important details - * @return the number of nanoseconds of cpu consumed by this thread since {@code Thread.start()}. + * @param context the context to reset */ - private long getThreadTotalCpuNs() { + public static void reset(String context) { if (THREAD_MX_BEAN != null) { - return THREAD_MX_BEAN.getCurrentThreadCpuTime(); - } else { - return UNSUPPORTED; + threadLocalTimer + .get() + .computeIfAbsent( + context, (ctx) -> new AtomicLong(THREAD_MX_BEAN.getCurrentThreadCpuTime())) + .set(THREAD_MX_BEAN.getCurrentThreadCpuTime()); } } + public static Optional readMSandReset(String context) { + return readNSAndReset(context) + .map((cpuTimeNs) -> TimeUnit.MILLISECONDS.convert(cpuTimeNs, TimeUnit.NANOSECONDS)); + } + /** - * Get the CPU usage information for the current thread since it created this {@link - * ThreadCpuTimer}. The result is undefined if called by any other thread. - * - * @return the thread's cpu since the creation of this {@link ThreadCpuTimer} instance. If the - * VM's cpu tracking is disabled, returned value will be {@link #UNSUPPORTED}. + * Cleanup method. This should be called at the very end of a request thread when it's absolutely + * sure no code will attempt a new reading. */ - public Optional getElapsedCpuMs() { - long cpuTimeNs = getElapsedCpuNs(); - return cpuTimeNs != UNSUPPORTED - ? Optional.of(TimeUnit.MILLISECONDS.convert(cpuTimeNs, TimeUnit.NANOSECONDS)) - : Optional.empty(); + public static void reset() { + threadLocalTimer.get().clear(); } @Override public String toString() { - return getElapsedCpuMs().map(String::valueOf).orElse("UNSUPPORTED"); + return THREAD_MX_BEAN == null + ? "UNSUPPORTED" + : "Timing contexts:" + threadLocalTimer.get().toString(); } } diff --git a/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java b/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java index 0d53b8653db..96a6bd5ee93 100644 --- a/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java +++ b/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java @@ -22,7 +22,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import org.apache.lucene.index.QueryTimeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +31,7 @@ * and optionally a method name, e.g. MoreLikeThisComponent or * ClusteringComponent.finishStage. */ -public class CallerSpecificQueryLimit implements QueryTimeout { +public class CallerSpecificQueryLimit implements QueryLimit { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); final StackWalker stackWalker = @@ -102,4 +101,9 @@ public boolean shouldExit() { } return matchingExpr.isPresent(); } + + @Override + public Object currentValue() { + return "This class just for testing, not a real limit"; + } } diff --git a/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java b/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java index ea02e448da8..1fd6c87616d 100644 --- a/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java +++ b/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java @@ -20,12 +20,16 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.cloud.CloudUtil; import org.apache.solr.cloud.SolrCloudTestCase; +import org.apache.solr.index.NoMergePolicyFactory; +import org.apache.solr.util.TestInjection; import org.apache.solr.util.ThreadCpuTimer; +import org.junit.AfterClass; import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; @@ -61,7 +65,12 @@ private static Path createConfigSet() throws Exception { } @BeforeClass - public static void setup() throws Exception { + public static void setupClass() throws Exception { + // Using NoMergePolicy and 100 commits we should get 100 segments (across all shards). + // At this point of writing MAX_SEGMENTS_PER_SLICE in lucene is 5, so we should be + // ensured that any multithreaded testing will create 20 executable tasks for the + // executor that was provided to index-searcher. + systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName()); System.setProperty(ThreadCpuTimer.ENABLE_CPU_TIME, "true"); Path configset = createConfigSet(); configureCluster(1).addConfig("conf", configset).configure(); @@ -73,8 +82,14 @@ public static void setup() throws Exception { cluster.getOpenOverseer().getSolrCloudManager(), "active", COLLECTION, clusterShape(3, 6)); for (int j = 0; j < 100; j++) { solrClient.add(COLLECTION, sdoc("id", "id-" + j, "val_i", j % 5)); + solrClient.commit(COLLECTION); // need to commit every doc to create many segments. } - solrClient.commit(COLLECTION); + } + + @AfterClass + public static void tearDownClass() { + TestInjection.cpuTimerDelayInjectedNS = null; + systemClearPropertySolrTestsMergePolicyFactory(); } @Test @@ -131,7 +146,9 @@ public void testDistribLimit() throws Exception { Number qtime = (Number) rsp.getHeader().get("QTime"); assertTrue("QTime expected " + qtime + " >> " + sleepMs, qtime.longValue() > sleepMs); assertNull("should not have partial results", rsp.getHeader().get("partialResults")); - + TestInjection.measureCpu(); + // 25 ms per 5 segments ~175ms each shard + TestInjection.cpuTimerDelayInjectedNS = new AtomicInteger(25_000_000); // timeAllowed set, should return partial results log.info("--- timeAllowed, partial results ---"); rsp = @@ -146,6 +163,28 @@ public void testDistribLimit() throws Exception { String.valueOf(sleepMs), "stages", "prepare,process", + "multiThreaded", + "false", + "timeAllowed", + "500")); + // System.err.println("rsp=" + rsp.jsonStr()); + assertNotNull("should have partial results", rsp.getHeader().get("partialResults")); + + log.info("--- timeAllowed, partial results, multithreading ---"); + rsp = + solrClient.query( + COLLECTION, + params( + "q", + "id:*", + "sort", + "id asc", + ExpensiveSearchComponent.SLEEP_MS_PARAM, + String.valueOf(sleepMs), + "stages", + "prepare,process", + "multiThreaded", + "true", "timeAllowed", "500")); // System.err.println("rsp=" + rsp.jsonStr()); @@ -161,15 +200,12 @@ public void testDistribLimit() throws Exception { "id:*", "sort", "id desc", - ExpensiveSearchComponent.CPU_LOAD_COUNT_PARAM, - "1", "stages", "prepare,process", "cpuAllowed", - "1000")); + "10000")); // System.err.println("rsp=" + rsp.jsonStr()); assertNull("should have full results", rsp.getHeader().get("partialResults")); - // cpuAllowed set, should return partial results log.info("--- cpuAllowed 1, partial results ---"); rsp = @@ -180,19 +216,17 @@ public void testDistribLimit() throws Exception { "id:*", "sort", "id desc", - ExpensiveSearchComponent.CPU_LOAD_COUNT_PARAM, - "10", "stages", "prepare,process", "cpuAllowed", - "50", + "100", "multiThreaded", "false")); // System.err.println("rsp=" + rsp.jsonStr()); assertNotNull("should have partial results", rsp.getHeader().get("partialResults")); // cpuAllowed set, should return partial results - log.info("--- cpuAllowed 2, partial results ---"); + log.info("--- cpuAllowed 2, partial results, multi-threaded ---"); rsp = solrClient.query( COLLECTION, @@ -201,14 +235,12 @@ public void testDistribLimit() throws Exception { "id:*", "sort", "id desc", - ExpensiveSearchComponent.CPU_LOAD_COUNT_PARAM, - "10", "stages", "prepare,process", "cpuAllowed", - "50", + "100", "multiThreaded", - "false")); + "true")); // System.err.println("rsp=" + rsp.jsonStr()); assertNotNull("should have partial results", rsp.getHeader().get("partialResults")); } diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java index a0444ea5a53..c917438249a 100644 --- a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java +++ b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java @@ -202,14 +202,15 @@ public static ExecutorService newMDCAwareFixedThreadPool( } public static ExecutorService newMDCAwareFixedThreadPool( - int nThreads, int queueCapacity, ThreadFactory threadFactory) { + int nThreads, int queueCapacity, ThreadFactory threadFactory, Runnable beforeExecute) { return new MDCAwareThreadPoolExecutor( nThreads, nThreads, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(queueCapacity), - threadFactory); + threadFactory, + beforeExecute); } /** @@ -257,8 +258,10 @@ public static ExecutorService newMDCAwareCachedThreadPool( public static class MDCAwareThreadPoolExecutor extends ThreadPoolExecutor { private static final int MAX_THREAD_NAME_LEN = 512; + public static final Runnable NOOP = () -> {}; private final boolean enableSubmitterStackTrace; + private final Runnable beforeExecuteTask; public MDCAwareThreadPoolExecutor( int corePoolSize, @@ -270,6 +273,7 @@ public MDCAwareThreadPoolExecutor( RejectedExecutionHandler handler) { super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler); this.enableSubmitterStackTrace = true; + this.beforeExecuteTask = NOOP; } public MDCAwareThreadPoolExecutor( @@ -280,6 +284,7 @@ public MDCAwareThreadPoolExecutor( BlockingQueue workQueue) { super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue); this.enableSubmitterStackTrace = true; + this.beforeExecuteTask = NOOP; } public MDCAwareThreadPoolExecutor( @@ -289,7 +294,8 @@ public MDCAwareThreadPoolExecutor( TimeUnit unit, BlockingQueue workQueue, ThreadFactory threadFactory) { - this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, true); + this( + corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, true, NOOP); } public MDCAwareThreadPoolExecutor( @@ -299,9 +305,30 @@ public MDCAwareThreadPoolExecutor( TimeUnit unit, BlockingQueue workQueue, ThreadFactory threadFactory, - boolean enableSubmitterStackTrace) { + Runnable beforeExecuteTask) { + this( + corePoolSize, + maximumPoolSize, + keepAliveTime, + unit, + workQueue, + threadFactory, + true, + beforeExecuteTask); + } + + public MDCAwareThreadPoolExecutor( + int corePoolSize, + int maximumPoolSize, + long keepAliveTime, + TimeUnit unit, + BlockingQueue workQueue, + ThreadFactory threadFactory, + boolean enableSubmitterStackTrace, + Runnable beforeExecuteTask) { super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory); this.enableSubmitterStackTrace = enableSubmitterStackTrace; + this.beforeExecuteTask = beforeExecuteTask; } public MDCAwareThreadPoolExecutor( @@ -313,6 +340,37 @@ public MDCAwareThreadPoolExecutor( RejectedExecutionHandler handler) { super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, handler); this.enableSubmitterStackTrace = true; + this.beforeExecuteTask = NOOP; + } + + public MDCAwareThreadPoolExecutor( + int corePoolSize, + int maximumPoolSize, + int keepAliveTime, + TimeUnit timeUnit, + BlockingQueue blockingQueue, + SolrNamedThreadFactory httpShardExecutor, + boolean enableSubmitterStackTrace) { + super( + corePoolSize, maximumPoolSize, keepAliveTime, timeUnit, blockingQueue, httpShardExecutor); + this.enableSubmitterStackTrace = enableSubmitterStackTrace; + this.beforeExecuteTask = NOOP; + } + + public MDCAwareThreadPoolExecutor( + int i, + int maxValue, + long l, + TimeUnit timeUnit, + BlockingQueue es, + SolrNamedThreadFactory testExecutor, + boolean b) { + this(i, maxValue, l, timeUnit, es, testExecutor, b, NOOP); + } + + @Override + protected void beforeExecute(Thread t, Runnable r) { + this.beforeExecuteTask.run(); } @Override From c625d13e04b2921f324499c1b5c88e39920b4e11 Mon Sep 17 00:00:00 2001 From: Jason Gerlowski Date: Tue, 30 Jul 2024 09:15:56 -0400 Subject: [PATCH 07/38] Produce test report for integration tests (#2600) BATS supports outputing test-reports in various formats (including the format used by our Java/JUnit tests), but our build wasn't taking advantage of this. This commit tweaks our BATS invocation so that a JUnit-style XML test report ('report.xml') is produced in the packaging module's 'build/test-output' directory. --- solr/packaging/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/solr/packaging/build.gradle b/solr/packaging/build.gradle index bf816677a02..25d17ea1125 100644 --- a/solr/packaging/build.gradle +++ b/solr/packaging/build.gradle @@ -294,7 +294,7 @@ class BatsTask extends Exec { // Note: tests to run must be listed after all other arguments // Additional debugging output: -x, --verbose-run - setArgs(['-T', '--print-output-on-failure'] + (testFiles.empty ? testDir : testFiles)) + setArgs(['-T', '--print-output-on-failure', '--report-formatter', 'junit', '--output', "$project.buildDir/test-output"] + (testFiles.empty ? testDir : testFiles)) super.exec() From 8ab0278644e2c2731faae9f43c50c959bbe2f712 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 30 Jul 2024 14:23:54 +0100 Subject: [PATCH 08/38] SOLR-13350, SOLR-17298: multi-threaded search: revert TestFiltering.java change (#2589) --- .../org/apache/solr/search/TestFiltering.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/search/TestFiltering.java b/solr/core/src/test/org/apache/solr/search/TestFiltering.java index e9194e170f9..5d9ad6bb9da 100644 --- a/solr/core/src/test/org/apache/solr/search/TestFiltering.java +++ b/solr/core/src/test/org/apache/solr/search/TestFiltering.java @@ -22,7 +22,6 @@ import java.util.Locale; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Query; -import org.apache.lucene.util.Bits; import org.apache.lucene.util.FixedBitSet; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.SolrInputDocument; @@ -93,14 +92,14 @@ public void testLiveDocsSharing() throws Exception { QueryResult res = new QueryResult(); searcher.search(res, cmd); set = res.getDocSet(); - assertEffectivelySame(set.getFixedBitSet(), live.getFixedBitSet()); + assertSame(set, live); cmd.setQuery(QParser.getParser(qstr + " OR id:0", null, req).getQuery()); cmd.setFilterList(QParser.getParser(qstr + " OR id:1", null, req).getQuery()); res = new QueryResult(); searcher.search(res, cmd); set = res.getDocSet(); - assertEffectivelySame(set.getFixedBitSet(), live.getFixedBitSet()); + assertSame(set, live); } } finally { @@ -108,19 +107,6 @@ public void testLiveDocsSharing() throws Exception { } } - /** If the a XOR b == 0, then both a & b are effectively the same bitset */ - private void assertEffectivelySame(FixedBitSet a, FixedBitSet b) { - FixedBitSet xor = a.clone(); - xor.xor(b); - assertEquals(new FixedBitSet(xor.length()), xor); - } - - private String bitsString(Bits bits) { - StringBuilder s = new StringBuilder(); - for (int i = 0; i < bits.length(); i++) s.append(bits.get(i) ? 1 : 0); - return s.toString(); - } - public void testCaching() throws Exception { clearIndex(); assertU(adoc("id", "4", "val_i", "1")); From c7630fe13ff9e5475c0c358beb9e92c12826e599 Mon Sep 17 00:00:00 2001 From: Jason Gerlowski Date: Tue, 30 Jul 2024 13:59:14 -0400 Subject: [PATCH 09/38] Default to MockDirectoryFactory in test configs (#2598) This was already the effective default value, by virtue of it being set in 'randomization.gradle'. But setting the value there instead of in the config files themselves meant that the value wasn't being properly set for IDE test runners. This commit fixes this by removing the 'randomization.gradle' setting and instead setting MockDirectoryFactory as the default (via ${} expansion syntax) in each individual test solrconfig file. --- gradle/testing/randomization.gradle | 1 - .../test-files/solr/collection1/conf/bad-error-solrconfig.xml | 2 +- .../test-files/solr/collection1/conf/bad-mpf-solrconfig.xml | 2 +- .../src/test-files/solr/collection1/conf/bad_solrconfig.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-SOLR-749.xml | 2 +- .../solrconfig-add-schema-fields-update-processor-chains.xml | 2 +- .../solr/collection1/conf/solrconfig-analytics-query.xml | 2 +- .../src/test-files/solr/collection1/conf/solrconfig-basic.xml | 2 +- .../solr/collection1/conf/solrconfig-cache-enable-disable.xml | 2 +- .../solr/collection1/conf/solrconfig-classification.xml | 2 +- .../solr/collection1/conf/solrconfig-collapseqparser.xml | 2 +- .../solr/collection1/conf/solrconfig-components-name.xml | 2 +- .../collection1/conf/solrconfig-concurrentmergescheduler.xml | 2 +- .../solr/collection1/conf/solrconfig-coreproperties.xml | 2 +- .../solr/collection1/conf/solrconfig-delaying-component.xml | 2 +- .../solr/collection1/conf/solrconfig-delpolicy1.xml | 2 +- .../solr/collection1/conf/solrconfig-delpolicy2.xml | 2 +- .../conf/solrconfig-distrib-update-processor-chains.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-elevate.xml | 2 +- .../collection1/conf/solrconfig-externalversionconstraint.xml | 2 +- .../solr/collection1/conf/solrconfig-follower-auth.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-follower.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-follower1.xml | 2 +- .../solr/collection1/conf/solrconfig-functionquery.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-headers.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-highlight.xml | 2 +- .../solr/collection1/conf/solrconfig-implicitproperties.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-leader.xml | 2 +- .../collection1/conf/solrconfig-leader1-keepOneBackup.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-leader1.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-leader2.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-leader3.xml | 2 +- .../collection1/conf/solrconfig-logmergepolicyfactory.xml | 2 +- .../solr/collection1/conf/solrconfig-lucene-codec.xml | 2 +- .../solr/collection1/conf/solrconfig-managed-schema-test.xml | 2 +- .../solr/collection1/conf/solrconfig-mergepolicy-defaults.xml | 2 +- .../solr/collection1/conf/solrconfig-mergepolicy-legacy.xml | 2 +- .../collection1/conf/solrconfig-mergepolicyfactory-nocfs.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-minhash.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-nocache.xml | 2 +- .../solr/collection1/conf/solrconfig-nomergepolicyfactory.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-noopregen.xml | 2 +- .../solr/collection1/conf/solrconfig-parallel-commit.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-paramset.xml | 2 +- .../conf/solrconfig-parsing-update-processor-chains.xml | 2 +- .../solr/collection1/conf/solrconfig-phrasesuggest.xml | 2 +- .../solr/collection1/conf/solrconfig-plugcollector.xml | 2 +- .../collection1/conf/solrconfig-pluggable-circuitbreaker.xml | 2 +- .../solr/collection1/conf/solrconfig-query-parser-init.xml | 2 +- .../solr/collection1/conf/solrconfig-querysender-noquery.xml | 2 +- .../solr/collection1/conf/solrconfig-querysender.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-repeater.xml | 2 +- .../solr/collection1/conf/solrconfig-replication-legacy.xml | 2 +- .../collection1/conf/solrconfig-response-log-component.xml | 2 +- .../solr/collection1/conf/solrconfig-searcher-listeners1.xml | 2 +- .../collection1/conf/solrconfig-sortingmergepolicyfactory.xml | 2 +- .../solr/collection1/conf/solrconfig-sortingresponse.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-spatial.xml | 2 +- .../solr/collection1/conf/solrconfig-spellcheckcomponent.xml | 2 +- .../solr/collection1/conf/solrconfig-spellchecker.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-tagger.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-test-misc.xml | 2 +- .../solr/collection1/conf/solrconfig-testxmlparser.xml | 2 +- .../collection1/conf/solrconfig-tieredmergepolicyfactory.xml | 2 +- .../src/test-files/solr/collection1/conf/solrconfig-tlog.xml | 2 +- .../solr/collection1/conf/solrconfig-transformers.xml | 2 +- .../collection1/conf/solrconfig-update-processor-chains.xml | 2 +- .../conf/solrconfig-warmer-randommergepolicyfactory.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-xinclude.xml | 2 +- solr/core/src/test-files/solr/collection1/conf/solrconfig.xml | 2 +- .../solr/collection1/conf/solrconfig_SimpleTextCodec.xml | 2 +- .../src/test-files/solr/collection1/conf/solrconfig_codec.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig_codec2.xml | 2 +- .../src/test-files/solr/collection1/conf/solrconfig_perf.xml | 2 +- .../solr/configsets/bad-mergepolicy/conf/solrconfig.xml | 2 +- .../solr/configsets/cloud-minimal/conf/solrconfig.xml | 2 +- .../solr/configsets/exitable-directory/conf/solrconfig.xml | 2 +- .../test-files/solr/configsets/minimal/conf/solrconfig.xml | 2 +- .../solr/collection1/conf/solrconfig-icucollate.xml | 2 +- .../solr/collection1/conf/solrconfig-opennlp-extract.xml | 2 +- .../clustering/solr/collection1/conf/solrconfig.xml | 2 +- .../extraction/solr/collection1/conf/solrconfig.xml | 2 +- .../solr/collection1/conf/solrconfig-languageidentifier.xml | 2 +- .../src/test-files/solr/collection1/conf/solrconfig-ltr.xml | 2 +- .../solr/collection1/conf/solrconfig-ltr_Th10_10.xml | 2 +- .../test-files/solr/collection1/conf/solrconfig-multiseg.xml | 2 +- .../collection1/conf/solrconfig-script-updateprocessor.xml | 2 +- .../conf/stateless-solrconfig-script-updateprocessor.xml | 2 +- .../solrj/solr/collection1/conf/solrconfig-follower1.xml | 2 +- .../solrj/solr/configsets/shared/conf/solrconfig.xml | 2 +- .../src/java/org/apache/solr/SolrTestCaseJ4.java | 4 ---- .../src/test-files/solr/collection1/conf/solrconfig.xml | 2 +- 92 files changed, 90 insertions(+), 95 deletions(-) diff --git a/gradle/testing/randomization.gradle b/gradle/testing/randomization.gradle index eb244a6ea21..9c809fc69e9 100644 --- a/gradle/testing/randomization.gradle +++ b/gradle/testing/randomization.gradle @@ -126,7 +126,6 @@ configure(allprojects.findAll {project -> project.path.startsWith(":solr") }) { plugins.withType(JavaPlugin) { ext { testOptions += [ - [propName: 'solr.directoryFactory', value: "org.apache.solr.core.MockDirectoryFactory", description: "Solr directory factory."], [propName: 'tests.src.home', value: null, description: "See SOLR-14023."], [propName: 'solr.tests.use.numeric.points', value: null, description: "Point implementation to use (true=numerics, false=trie)."], ] diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-error-solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/bad-error-solrconfig.xml index c8bb8cf66e3..af4edbddcaf 100644 --- a/solr/core/src/test-files/solr/collection1/conf/bad-error-solrconfig.xml +++ b/solr/core/src/test-files/solr/collection1/conf/bad-error-solrconfig.xml @@ -19,7 +19,7 @@ - + ${tests.luceneMatchVersion:LATEST} diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-mpf-solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/bad-mpf-solrconfig.xml index 19d786056ad..bfaf24a0cfa 100644 --- a/solr/core/src/test-files/solr/collection1/conf/bad-mpf-solrconfig.xml +++ b/solr/core/src/test-files/solr/collection1/conf/bad-mpf-solrconfig.xml @@ -19,7 +19,7 @@ - + ${tests.luceneMatchVersion:LATEST} diff --git a/solr/core/src/test-files/solr/collection1/conf/bad_solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/bad_solrconfig.xml index e24df5846fd..70edc2d4723 100644 --- a/solr/core/src/test-files/solr/collection1/conf/bad_solrconfig.xml +++ b/solr/core/src/test-files/solr/collection1/conf/bad_solrconfig.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${unset.sys.property} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml index 9c2411bde33..502146d9591 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml index 34ee4f7d038..d95c8a740c3 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml @@ -24,7 +24,7 @@ ${tests.luceneMatchVersion:LATEST} - + true diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml index cda45dfe364..b6d4a1cb673 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml index d98ecacf454..d28e6a6d052 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml @@ -23,7 +23,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml index d51f8050c56..96e0b6d754c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml index 3370600574d..038813e8dd3 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml @@ -24,7 +24,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml index 8d242f46a54..d104e8d80af 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml index 6a14a8cc91c..0ca0e186db6 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml @@ -29,7 +29,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-concurrentmergescheduler.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-concurrentmergescheduler.xml index 140c4cfeedc..3c7c7959ed0 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-concurrentmergescheduler.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-concurrentmergescheduler.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-coreproperties.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-coreproperties.xml index fc707d461fd..13bada25dd7 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-coreproperties.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-coreproperties.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml index cfb812fa25b..7cabbd19c6f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml index 424783beef5..2bd4d26b150 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml index bb4d3f85ba0..8a57b7fb8e1 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml index b23a7dc9ff6..9b56a39a52f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.hdfs.blockcache.enabled:true} ${solr.hdfs.blockcache.blocksperbank:1024} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml index d814a538a50..63bb8f84ffa 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml @@ -28,7 +28,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml index 0e785ce0adc..5e366166ea1 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.hdfs.blockcache.enabled:true} ${solr.hdfs.blockcache.blocksperbank:1024} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower-auth.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower-auth.xml index 1635cfb099b..236a0997b5a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower-auth.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower-auth.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml index fc0298ef5fd..f009ac9e59c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml index 71f4157ccb2..9ae3600b200 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml index ef0c39ae8e4..2086a6c4ba9 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml @@ -25,7 +25,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml index 328fc9bd56a..695cd36e2a4 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml index c714a4148d7..7688b3e0a7e 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml @@ -25,7 +25,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml index b9a72c5634a..072cae1d869 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml @@ -26,7 +26,7 @@ ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}"/> diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml index 89c06ad3a54..ea2e8e332da 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml index 36065b43ae7..40e93b8e5c5 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml index 4e4e3999224..e4afd9b9f8f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml index f6c2a77dde3..933337f0a8c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml index 49d1ed31b5a..b46fb393cf3 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml @@ -20,7 +20,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicyfactory.xml index 539fd5c4d5c..528ea879568 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicyfactory.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicyfactory.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-lucene-codec.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-lucene-codec.xml index 1ee427dc40d..9c131263850 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-lucene-codec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-lucene-codec.xml @@ -29,7 +29,7 @@ ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml index 666132f8e74..e3f4612e7e1 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml @@ -22,6 +22,6 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-defaults.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-defaults.xml index 3e0cf1927b6..76684eaebe8 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-defaults.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-defaults.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml index b67d6645f31..d02b9b01827 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicyfactory-nocfs.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicyfactory-nocfs.xml index b93fabd7c3a..f9276ba9298 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicyfactory-nocfs.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicyfactory-nocfs.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml index 9fa236dda0b..0d32632fc94 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml index fb891f822fd..f4ffe8a8502 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml @@ -24,7 +24,7 @@ - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nomergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nomergepolicyfactory.xml index 62fb05b03c4..6e5545be7c4 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nomergepolicyfactory.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nomergepolicyfactory.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml index e2a20a3379c..1799508b115 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parallel-commit.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parallel-commit.xml index 3e619948e18..662089c170a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parallel-commit.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parallel-commit.xml @@ -24,7 +24,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml index a9e71f63ebc..1b41d2b949a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml @@ -23,7 +23,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml index 43f2d285a9f..0416d3e796e 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml @@ -24,7 +24,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml index 0f79d7c4572..65f1e9f3a4a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml index 845998ec2f4..63f2ce29b7e 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml index 52956f60824..4c8d441b2d3 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-pluggable-circuitbreaker.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml index eb538fa620b..0dd520baca9 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml @@ -24,7 +24,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml index 9d4f83d68c3..e6e94a2aa67 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml @@ -23,7 +23,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml index 1404f8abb4f..f8bc376ad29 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml @@ -23,7 +23,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml index f5571f9827f..b426ac82f93 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml index 43c42ffb089..ddd116be38a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.data.dir:} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml index 643d9a62fc7..f3bb105bd25 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml @@ -31,7 +31,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml index c71f8baf9d4..ebceb148579 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml @@ -30,7 +30,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingmergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingmergepolicyfactory.xml index 5920348551a..5d1e7242e70 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingmergepolicyfactory.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingmergepolicyfactory.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml index 1bb3c97a225..ee27bec6144 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml @@ -23,7 +23,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml index 889d1c24968..f6cca32f8a4 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml index 0253d91b804..8c7387da9c0 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml @@ -44,7 +44,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml index a876fb6f27c..5349edc7c8c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml @@ -21,7 +21,7 @@ - + ${tests.luceneMatchVersion:LATEST} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tagger.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tagger.xml index c97ce085660..de0b3569270 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tagger.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tagger.xml @@ -26,7 +26,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml index 00d49db3285..1020db8319e 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml @@ -25,7 +25,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml index 401710e8d32..f9f8427707b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicyfactory.xml index 3494db00d9c..5a73fc9a152 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicyfactory.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicyfactory.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml index 5203031c456..aff143c616f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.hdfs.blockcache.enabled:true} ${solr.hdfs.blockcache.blocksperbank:1024} diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml index ef38e8099e9..b4e4846426d 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml @@ -23,7 +23,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml index e22ad695859..4e6d7bdd370 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml @@ -27,7 +27,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer-randommergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer-randommergepolicyfactory.xml index 21101b1d9e6..3c6056a3fc1 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer-randommergepolicyfactory.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer-randommergepolicyfactory.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml index 17df214cb71..48f6676645f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml @@ -21,7 +21,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml index 5632e36cb0f..2c06096732c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml index 7b0c3e368ba..a971665edee 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml index ad080961d9f..991d2a50f61 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml index c4a8ae73dfe..94f934dbcbe 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml @@ -19,7 +19,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml index f2b5ef8a381..0c6b57c4e77 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml @@ -28,7 +28,7 @@ - + diff --git a/solr/core/src/test-files/solr/configsets/bad-mergepolicy/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/bad-mergepolicy/conf/solrconfig.xml index 3ef080dcca3..c286b018cef 100644 --- a/solr/core/src/test-files/solr/configsets/bad-mergepolicy/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/configsets/bad-mergepolicy/conf/solrconfig.xml @@ -19,7 +19,7 @@ - + ${tests.luceneMatchVersion:LATEST} diff --git a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml index 853ba656241..baef69fde90 100644 --- a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml @@ -24,7 +24,7 @@ ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}"/> ${tests.luceneMatchVersion:LATEST} diff --git a/solr/core/src/test-files/solr/configsets/exitable-directory/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/exitable-directory/conf/solrconfig.xml index 515ad64d7f5..7e1f3434e96 100644 --- a/solr/core/src/test-files/solr/configsets/exitable-directory/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/configsets/exitable-directory/conf/solrconfig.xml @@ -22,7 +22,7 @@ ${tests.luceneMatchVersion:LATEST} - + ${solr.hdfs.blockcache.enabled:true} ${solr.hdfs.blockcache.blocksperbank:1024} diff --git a/solr/core/src/test-files/solr/configsets/minimal/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/minimal/conf/solrconfig.xml index 346b0448318..4e90243daf6 100644 --- a/solr/core/src/test-files/solr/configsets/minimal/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/configsets/minimal/conf/solrconfig.xml @@ -24,7 +24,7 @@ ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}"/> ${tests.luceneMatchVersion:LATEST} diff --git a/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-icucollate.xml b/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-icucollate.xml index 90c52d71cbe..44ef6530752 100644 --- a/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-icucollate.xml +++ b/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-icucollate.xml @@ -23,5 +23,5 @@ ${useCompoundFile:false} - + diff --git a/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml b/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml index 7fd793e94d3..1bf101f0db1 100644 --- a/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml +++ b/solr/modules/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml @@ -22,7 +22,7 @@ - + diff --git a/solr/modules/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml b/solr/modules/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml index 4d268b0efd8..7376c7a07d1 100644 --- a/solr/modules/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml +++ b/solr/modules/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml @@ -25,7 +25,7 @@ If replication is in use, this should match the replication configuration. --> ${solr.data.dir:} - + diff --git a/solr/modules/extraction/src/test-files/extraction/solr/collection1/conf/solrconfig.xml b/solr/modules/extraction/src/test-files/extraction/solr/collection1/conf/solrconfig.xml index b35a27fac39..2c52f4591e8 100644 --- a/solr/modules/extraction/src/test-files/extraction/solr/collection1/conf/solrconfig.xml +++ b/solr/modules/extraction/src/test-files/extraction/solr/collection1/conf/solrconfig.xml @@ -27,7 +27,7 @@ It defaults to "index" if not present, and should probably not be changed if replication is in use. --> ${solr.data.dir:} - + diff --git a/solr/modules/langid/src/test-files/langid/solr/collection1/conf/solrconfig-languageidentifier.xml b/solr/modules/langid/src/test-files/langid/solr/collection1/conf/solrconfig-languageidentifier.xml index f03387a170b..a951ea4d80d 100644 --- a/solr/modules/langid/src/test-files/langid/solr/collection1/conf/solrconfig-languageidentifier.xml +++ b/solr/modules/langid/src/test-files/langid/solr/collection1/conf/solrconfig-languageidentifier.xml @@ -31,7 +31,7 @@ - + ${tests.luceneMatchVersion:LATEST} diff --git a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml index 172a0bca6a8..b863d61728c 100644 --- a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml +++ b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml @@ -14,7 +14,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}" /> diff --git a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml index 7aca7e10db5..37ae68a2580 100644 --- a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml +++ b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml @@ -14,7 +14,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}" /> diff --git a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml index c5db8670234..911db9a9f55 100644 --- a/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml +++ b/solr/modules/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml @@ -14,7 +14,7 @@ ${tests.luceneMatchVersion:LATEST} ${solr.data.dir:} + class="${solr.directoryFactory:solr.MockDirectoryFactory}" /> diff --git a/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/solrconfig-script-updateprocessor.xml b/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/solrconfig-script-updateprocessor.xml index afa5a7c56d7..82ce0331eda 100644 --- a/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/solrconfig-script-updateprocessor.xml +++ b/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/solrconfig-script-updateprocessor.xml @@ -25,7 +25,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/stateless-solrconfig-script-updateprocessor.xml b/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/stateless-solrconfig-script-updateprocessor.xml index 58fbb862860..d8c6fd2bdbb 100644 --- a/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/stateless-solrconfig-script-updateprocessor.xml +++ b/solr/modules/scripting/src/test-files/scripting/solr/collection1/conf/stateless-solrconfig-script-updateprocessor.xml @@ -25,7 +25,7 @@ ${tests.luceneMatchVersion:LATEST} - + diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml index ab2773d6eb4..43ff0410a8c 100644 --- a/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml @@ -23,7 +23,7 @@ ${useCompoundFile:false} ${solr.data.dir:} - + diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/solrconfig.xml b/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/solrconfig.xml index 11ea52b8da2..fb89854b08c 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/solrconfig.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/shared/conf/solrconfig.xml @@ -26,7 +26,7 @@ ${useCompoundFile:false} ${tempDir}/data/${l10n:}-${version:} - + diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index 55ad19495e0..fc723f60aa5 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -769,10 +769,6 @@ public static void initCore() throws Exception { log.info("####initCore"); ignoreException("ignore_exception"); - factoryProp = System.getProperty("solr.directoryFactory"); - if (factoryProp == null) { - System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory"); - } // other methods like starting a jetty instance need these too System.setProperty("solr.test.sys.prop1", "propone"); diff --git a/solr/test-framework/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/test-framework/src/test-files/solr/collection1/conf/solrconfig.xml index 82dca6384d8..ea95918291c 100644 --- a/solr/test-framework/src/test-files/solr/collection1/conf/solrconfig.xml +++ b/solr/test-framework/src/test-files/solr/collection1/conf/solrconfig.xml @@ -38,7 +38,7 @@ - + 1000000 2000000 3000000 From 829dd2805e4bd9bbe7cfb5f01554e14ce9035036 Mon Sep 17 00:00:00 2001 From: Matthew Biscocho <54160956+mlbiscoc@users.noreply.github.com> Date: Tue, 30 Jul 2024 22:48:50 -0400 Subject: [PATCH 10/38] SOLR-17368: TestPrometheusResponseWriter redesign (#2586) Co-authored-by: mbiscocho --- .../prometheus/solr-prometheus-output.txt | 535 ------------------ .../TestPrometheusResponseWriter.java | 125 ++-- 2 files changed, 89 insertions(+), 571 deletions(-) delete mode 100644 solr/core/src/test-files/prometheus/solr-prometheus-output.txt diff --git a/solr/core/src/test-files/prometheus/solr-prometheus-output.txt b/solr/core/src/test-files/prometheus/solr-prometheus-output.txt deleted file mode 100644 index 825381fe4e8..00000000000 --- a/solr/core/src/test-files/prometheus/solr-prometheus-output.txt +++ /dev/null @@ -1,535 +0,0 @@ -# TYPE solr_metrics_jvm_buffers gauge -solr_metrics_jvm_buffers{item="Count",pool="direct"} -solr_metrics_jvm_buffers{item="Count",pool="mapped"} -solr_metrics_jvm_buffers{item="Count",pool="mapped - 'non-volatile memory'"} -# TYPE solr_metrics_jvm_buffers_bytes gauge -solr_metrics_jvm_buffers_bytes{item="MemoryUsed",pool="direct"} -solr_metrics_jvm_buffers_bytes{item="MemoryUsed",pool="mapped"} -solr_metrics_jvm_buffers_bytes{item="MemoryUsed",pool="mapped - 'non-volatile memory'"} -solr_metrics_jvm_buffers_bytes{item="TotalCapacity",pool="direct"} -solr_metrics_jvm_buffers_bytes{item="TotalCapacity",pool="mapped"} -solr_metrics_jvm_buffers_bytes{item="TotalCapacity",pool="mapped - 'non-volatile memory'"} -# TYPE solr_metrics_jvm_gc gauge -solr_metrics_jvm_gc{} -solr_metrics_jvm_gc{} -# TYPE solr_metrics_jvm_gc_seconds gauge -solr_metrics_jvm_gc_seconds{} -solr_metrics_jvm_gc_seconds{} -# TYPE solr_metrics_jvm_heap gauge -solr_metrics_jvm_heap{item="committed",memory="heap"} -solr_metrics_jvm_heap{item="committed",memory="non-heap"} -solr_metrics_jvm_heap{item="committed",memory="total"} -solr_metrics_jvm_heap{item="init",memory="heap"} -solr_metrics_jvm_heap{item="init",memory="non-heap"} -solr_metrics_jvm_heap{item="init",memory="total"} -solr_metrics_jvm_heap{item="max",memory="heap"} -solr_metrics_jvm_heap{item="max",memory="non-heap"} -solr_metrics_jvm_heap{item="max",memory="total"} -solr_metrics_jvm_heap{item="usage",memory="heap"} -solr_metrics_jvm_heap{item="usage",memory="non-heap"} -solr_metrics_jvm_heap{item="used",memory="heap"} -solr_metrics_jvm_heap{item="used",memory="non-heap"} -solr_metrics_jvm_heap{item="used",memory="total"} -# TYPE solr_metrics_jvm_memory_pools_bytes gauge -solr_metrics_jvm_memory_pools_bytes{item="committed",space="CodeCache"} -solr_metrics_jvm_memory_pools_bytes{item="committed",space="Compressed-Class-Space"} -solr_metrics_jvm_memory_pools_bytes{item="committed",space="Metaspace"} -solr_metrics_jvm_memory_pools_bytes{item="committed",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="committed",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="committed",space="PS-Survivor-Space"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="CodeCache"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="Compressed-Class-Space"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="Metaspace"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="init",space="PS-Survivor-Space"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="CodeCache"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="Compressed-Class-Space"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="Metaspace"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="max",space="PS-Survivor-Space"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="CodeCache"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="Compressed-Class-Space"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="Metaspace"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="usage",space="PS-Survivor-Space"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="CodeCache"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="Compressed-Class-Space"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="Metaspace"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="used",space="PS-Survivor-Space"} -solr_metrics_jvm_memory_pools_bytes{item="used-after-gc",space="PS-Eden-Space"} -solr_metrics_jvm_memory_pools_bytes{item="used-after-gc",space="PS-Old-Gen"} -solr_metrics_jvm_memory_pools_bytes{item="used-after-gc",space="PS-Survivor-Space"} -# TYPE solr_metrics_jvm_threads gauge -solr_metrics_jvm_threads{item="blocked"} -solr_metrics_jvm_threads{item="count"} -solr_metrics_jvm_threads{item="daemon"} -solr_metrics_jvm_threads{item="deadlock"} -solr_metrics_jvm_threads{item="new"} -solr_metrics_jvm_threads{item="peak"} -solr_metrics_jvm_threads{item="runnable"} -solr_metrics_jvm_threads{item="terminated"} -solr_metrics_jvm_threads{item="timed_waiting"} -solr_metrics_jvm_threads{item="total_started"} -solr_metrics_jvm_threads{item="waiting"} -# TYPE solr_metrics_os gauge -solr_metrics_os{item="availableProcessors"} -solr_metrics_os{item="committedVirtualMemorySize"} -solr_metrics_os{item="cpuLoad"} -solr_metrics_os{item="freeMemorySize"} -solr_metrics_os{item="freePhysicalMemorySize"} -solr_metrics_os{item="freeSwapSpaceSize"} -solr_metrics_os{item="maxFileDescriptorCount"} -solr_metrics_os{item="openFileDescriptorCount"} -solr_metrics_os{item="processCpuLoad"} -solr_metrics_os{item="processCpuTime"} -solr_metrics_os{item="systemCpuLoad"} -solr_metrics_os{item="systemLoadAverage"} -solr_metrics_os{item="totalMemorySize"} -solr_metrics_os{item="totalPhysicalMemorySize"} -solr_metrics_os{item="totalSwapSpaceSize"} -# TYPE solr_metrics_node_connections gauge -solr_metrics_node_connections{category="UPDATE",handler="updateShardHandler",type="availableConnections"} -solr_metrics_node_connections{category="UPDATE",handler="updateShardHandler",type="leasedConnections"} -solr_metrics_node_connections{category="UPDATE",handler="updateShardHandler",type="maxConnections"} -solr_metrics_node_connections{category="UPDATE",handler="updateShardHandler",type="pendingConnections"} -# TYPE solr_metrics_node_core_root_fs_bytes gauge -solr_metrics_node_core_root_fs_bytes{category="CONTAINER",item="totalSpace"} -solr_metrics_node_core_root_fs_bytes{category="CONTAINER",item="usableSpace"} -# TYPE solr_metrics_node_cores gauge -solr_metrics_node_cores{category="CONTAINER",item="lazy"} -solr_metrics_node_cores{category="CONTAINER",item="loaded"} -solr_metrics_node_cores{category="CONTAINER",item="unloaded"} -# TYPE solr_metrics_node_requests_total counter -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/authorization",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/authorization",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/authorization",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/authorization",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/authorization",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/collections",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/collections",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/collections",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/collections",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/collections",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/configs",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/configs",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/configs",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/configs",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/configs",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/cores",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/cores",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/cores",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/cores",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/cores",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/info",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/info",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/info",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/info",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/info",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/metrics",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/metrics",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/metrics",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/metrics",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/metrics",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper",type="timeouts"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper/status",type="clientErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper/status",type="errors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper/status",type="requests"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper/status",type="serverErrors"} -solr_metrics_node_requests_total{category="ADMIN",handler="/admin/zookeeper/status",type="timeouts"} -# TYPE solr_metrics_node_requests_time_total counter -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/authorization"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/collections"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/configs"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/cores"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/info"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/metrics"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/zookeeper"} -solr_metrics_node_requests_time_total{category="ADMIN",handler="/admin/zookeeper/status"} -# TYPE solr_metrics_node_thread_pool_total counter -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreAdminExecutor",handler="/admin/cores",task="completed"} -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreAdminExecutor",handler="/admin/cores",task="running"} -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreAdminExecutor",handler="/admin/cores",task="submitted"} -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreExpensiveAdminExecutor",handler="/admin/cores",task="completed"} -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreExpensiveAdminExecutor",handler="/admin/cores",task="running"} -solr_metrics_node_thread_pool_total{category="ADMIN",executer="parallelCoreExpensiveAdminExecutor",handler="/admin/cores",task="submitted"} -solr_metrics_node_thread_pool_total{category="QUERY",executer="httpShardExecutor",handler="httpShardHandler",task="completed"} -solr_metrics_node_thread_pool_total{category="QUERY",executer="httpShardExecutor",handler="httpShardHandler",task="running"} -solr_metrics_node_thread_pool_total{category="QUERY",executer="httpShardExecutor",handler="httpShardHandler",task="submitted"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="recoveryExecutor",handler="updateShardHandler",task="completed"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="recoveryExecutor",handler="updateShardHandler",task="running"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="recoveryExecutor",handler="updateShardHandler",task="submitted"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="updateOnlyExecutor",handler="updateShardHandler",task="completed"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="updateOnlyExecutor",handler="updateShardHandler",task="running"} -solr_metrics_node_thread_pool_total{category="UPDATE",executer="updateOnlyExecutor",handler="updateShardHandler",task="submitted"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreContainerWorkExecutor",task="completed"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreContainerWorkExecutor",task="running"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreContainerWorkExecutor",task="submitted"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreLoadExecutor",task="completed"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreLoadExecutor",task="running"} -solr_metrics_node_thread_pool_total{category="CONTAINER",executer="coreLoadExecutor",task="submitted"} -# TYPE solr_metrics_core_average_request_time gauge -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/file"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/luke"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/mbeans"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/ping"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/plugins"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/segments"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/admin/system"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/config"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/schema"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/tasks/cancel"} -solr_metrics_core_average_request_time{category="ADMIN",core="collection1",handler="/tasks/list"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/debug/dump"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/export"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/export[shard]"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/get"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/get[shard]"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/graph"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/query"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/query[shard]"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/select"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/select[shard]"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/stream"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/terms"} -solr_metrics_core_average_request_time{category="QUERY",core="collection1",handler="/terms[shard]"} -solr_metrics_core_average_request_time{category="REPLICATION",core="collection1",handler="/replication"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="/update"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="/update/cbor"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="/update/csv"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="/update/json"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="/update/json/docs"} -solr_metrics_core_average_request_time{category="UPDATE",core="collection1",handler="update"} -# TYPE solr_metrics_core_average_searcher_warmup_time gauge -solr_metrics_core_average_searcher_warmup_time{core="collection1",type="time"} -solr_metrics_core_average_searcher_warmup_time{core="collection1",type="warmup"} -# TYPE solr_metrics_core_cache gauge -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="cumulative_evictions"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="cumulative_hitratio"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="cumulative_hits"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="cumulative_inserts"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="cumulative_lookups"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="evictions"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="hitratio"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="hits"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="inserts"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="maxRamMB"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="ramBytesUsed"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="size"} -solr_metrics_core_cache{cacheType="documentCache",core="collection1",item="warmupTime"} -solr_metrics_core_cache{cacheType="fieldCache",core="collection1",item="entries_count"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="cumulative_evictions"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="cumulative_hitratio"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="cumulative_hits"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="cumulative_inserts"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="cumulative_lookups"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="evictions"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="hitratio"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="hits"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="inserts"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="maxRamMB"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="ramBytesUsed"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="size"} -solr_metrics_core_cache{cacheType="fieldValueCache",core="collection1",item="warmupTime"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="cumulative_evictions"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="cumulative_hitratio"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="cumulative_hits"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="cumulative_inserts"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="cumulative_lookups"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="evictions"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="hitratio"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="hits"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="inserts"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="maxRamMB"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="ramBytesUsed"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="size"} -solr_metrics_core_cache{cacheType="filterCache",core="collection1",item="warmupTime"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="cumulative_evictions"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="cumulative_hitratio"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="cumulative_hits"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="cumulative_inserts"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="cumulative_lookups"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="evictions"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="hitratio"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="hits"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="inserts"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="maxRamMB"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="ramBytesUsed"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="size"} -solr_metrics_core_cache{cacheType="perSegFilter",core="collection1",item="warmupTime"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="cumulative_evictions"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="cumulative_hitratio"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="cumulative_hits"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="cumulative_inserts"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="cumulative_lookups"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="evictions"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="hitratio"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="hits"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="inserts"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="maxRamMB"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="ramBytesUsed"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="size"} -solr_metrics_core_cache{cacheType="queryResultCache",core="collection1",item="warmupTime"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="lookups"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="mergeToGlobalStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="missingGlobalFieldStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="missingGlobalTermStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="receiveGlobalStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="retrieveStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="returnLocalStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="sendGlobalStats"} -solr_metrics_core_cache{cacheType="statsCache",core="collection1",item="useCachedGlobalStats"} -solr_metrics_core_cache{core="collection1",item="hits",type="liveDocsCache"} -solr_metrics_core_cache{core="collection1",item="inserts",type="liveDocsCache"} -solr_metrics_core_cache{core="collection1",item="naiveHits",type="liveDocsCache"} -# TYPE solr_metrics_core_highlighter_requests_total counter -solr_metrics_core_highlighter_requests_total{core="collection1",item="breakIterator",type="SolrBoundaryScanner"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="colored",type="SolrFragmentsBuilder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrBoundaryScanner"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrEncoder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrFormatter"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrFragListBuilder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrFragmenter"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="default",type="SolrFragmentsBuilder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="gap",type="SolrFragmenter"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="html",type="SolrEncoder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="html",type="SolrFormatter"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="regex",type="SolrFragmenter"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="simple",type="SolrFragListBuilder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="single",type="SolrFragListBuilder"} -solr_metrics_core_highlighter_requests_total{core="collection1",item="weighted",type="SolrFragListBuilder"} -# TYPE solr_metrics_core_index_size_bytes gauge -solr_metrics_core_index_size_bytes{core="collection1"} -# TYPE solr_metrics_core_requests_total counter -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/file",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/file",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/file",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/file",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/file",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/luke",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/luke",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/luke",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/luke",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/luke",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/mbeans",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/mbeans",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/mbeans",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/mbeans",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/mbeans",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/ping",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/ping",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/ping",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/ping",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/ping",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/plugins",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/plugins",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/plugins",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/plugins",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/plugins",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/segments",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/segments",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/segments",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/segments",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/segments",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/system",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/system",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/system",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/system",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/admin/system",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/config",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/config",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/config",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/config",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/config",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/schema",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/schema",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/schema",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/schema",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/schema",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/cancel",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/cancel",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/cancel",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/cancel",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/cancel",type="timeouts"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/list",type="clientErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/list",type="errors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/list",type="requests"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/list",type="serverErrors"} -solr_metrics_core_requests_total{category="ADMIN",core="collection1",handler="/tasks/list",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/debug/dump",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/debug/dump",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/debug/dump",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/debug/dump",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/debug/dump",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export[shard]",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export[shard]",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export[shard]",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export[shard]",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/export[shard]",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get[shard]",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get[shard]",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get[shard]",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get[shard]",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/get[shard]",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/graph",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/graph",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/graph",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/graph",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/graph",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query[shard]",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query[shard]",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query[shard]",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query[shard]",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/query[shard]",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select[shard]",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select[shard]",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select[shard]",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select[shard]",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/select[shard]",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/stream",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/stream",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/stream",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/stream",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/stream",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms",type="timeouts"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms[shard]",type="clientErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms[shard]",type="errors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms[shard]",type="requests"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms[shard]",type="serverErrors"} -solr_metrics_core_requests_total{category="QUERY",core="collection1",handler="/terms[shard]",type="timeouts"} -solr_metrics_core_requests_total{category="REPLICATION",core="collection1",handler="/replication",type="clientErrors"} -solr_metrics_core_requests_total{category="REPLICATION",core="collection1",handler="/replication",type="errors"} -solr_metrics_core_requests_total{category="REPLICATION",core="collection1",handler="/replication",type="requests"} -solr_metrics_core_requests_total{category="REPLICATION",core="collection1",handler="/replication",type="serverErrors"} -solr_metrics_core_requests_total{category="REPLICATION",core="collection1",handler="/replication",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/cbor",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/cbor",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/cbor",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/cbor",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/cbor",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/csv",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/csv",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/csv",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/csv",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/csv",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json/docs",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json/docs",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json/docs",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json/docs",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="/update/json/docs",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="update",type="clientErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="update",type="errors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="update",type="requests"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="update",type="serverErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="update",type="timeouts"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="commits"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="cumulativeAdds"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="cumulativeDeletesById"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="cumulativeDeletesByQuery"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="cumulativeErrors"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="expungeDeletes"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="merges"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="optimizes"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="rollbacks"} -solr_metrics_core_requests_total{category="UPDATE",core="collection1",handler="updateHandler",type="splits"} -# TYPE solr_metrics_core_requests_time_total counter -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/file"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/luke"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/mbeans"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/ping"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/plugins"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/segments"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/admin/system"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/config"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/schema"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/tasks/cancel"} -solr_metrics_core_requests_time_total{category="ADMIN",core="collection1",handler="/tasks/list"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/debug/dump"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/export"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/export[shard]"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/get"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/get[shard]"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/graph"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/query"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/query[shard]"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/select"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/select[shard]"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/stream"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/terms"} -solr_metrics_core_requests_time_total{category="QUERY",core="collection1",handler="/terms[shard]"} -solr_metrics_core_requests_time_total{category="REPLICATION",core="collection1",handler="/replication"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="/update"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="/update/cbor"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="/update/csv"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="/update/json"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="/update/json/docs"} -solr_metrics_core_requests_time_total{category="UPDATE",core="collection1",handler="update"} -# TYPE solr_metrics_core_searcher_documents gauge -solr_metrics_core_searcher_documents{core="collection1",type="deletedDocs"} -solr_metrics_core_searcher_documents{core="collection1",type="fullSortCount"} -solr_metrics_core_searcher_documents{core="collection1",type="indexCommitSize"} -solr_metrics_core_searcher_documents{core="collection1",type="indexVersion"} -solr_metrics_core_searcher_documents{core="collection1",type="maxDoc"} -solr_metrics_core_searcher_documents{core="collection1",type="numDocs"} -solr_metrics_core_searcher_documents{core="collection1",type="skipSortCount"} -solr_metrics_core_searcher_documents{core="collection1",type="warmupTime"} -# TYPE solr_metrics_core_tlog_total counter -solr_metrics_core_tlog_total{core="collection1",item="applyingBuffered"} -solr_metrics_core_tlog_total{core="collection1",item="copyOverOldUpdates"} -solr_metrics_core_tlog_total{core="collection1",item="replay"} -# TYPE solr_metrics_core_update_handler gauge -solr_metrics_core_update_handler{category="REPLICATION",core="collection1",handler="/replication",type="generation"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="adds"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="autoCommits"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="deletesById"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="deletesByQuery"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="docsPending"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="errors"} -solr_metrics_core_update_handler{category="UPDATE",core="collection1",handler="updateHandler",type="softAutoCommits"} diff --git a/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java index c183a2610f6..8cf651e3928 100644 --- a/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java +++ b/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java @@ -16,9 +16,14 @@ */ package org.apache.solr.response; +import com.codahale.metrics.Counter; import com.codahale.metrics.Gauge; +import com.codahale.metrics.Meter; import com.codahale.metrics.SettableGauge; -import java.nio.file.Files; +import java.lang.invoke.MethodHandles; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.client.solrj.SolrClient; @@ -26,65 +31,113 @@ import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; -import org.apache.solr.embedded.JettySolrRunner; import org.apache.solr.metrics.SolrMetricManager; import org.apache.solr.util.ExternalPaths; import org.apache.solr.util.SolrJettyTestRule; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -/** Tests the {@link PrometheusResponseWriter} behavior */ -@LuceneTestCase.AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-17368") public class TestPrometheusResponseWriter extends SolrTestCaseJ4 { - @ClassRule public static SolrJettyTestRule solrClientTestRule = new SolrJettyTestRule(); - public static JettySolrRunner jetty; + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + public static final List VALID_PROMETHEUS_VALUES = Arrays.asList("NaN", "+Inf", "-Inf"); @BeforeClass public static void beforeClass() throws Exception { solrClientTestRule.startSolr(LuceneTestCase.createTempDir()); - jetty = solrClientTestRule.getJetty(); solrClientTestRule.newCollection().withConfigSet(ExternalPaths.DEFAULT_CONFIGSET).create(); - jetty.getCoreContainer().waitForLoadingCoresToFinish(30000); - // Manually register metrics not initializing from JettyTestRule - registerGauge( - jetty.getCoreContainer().getMetricManager(), - "solr.jvm", - "buffers.mapped - 'non-volatile memory'.Count"); - registerGauge( - jetty.getCoreContainer().getMetricManager(), - "solr.jvm", - "buffers.mapped - 'non-volatile memory'.MemoryUsed"); - registerGauge( - jetty.getCoreContainer().getMetricManager(), - "solr.jvm", - "buffers.mapped - 'non-volatile memory'.TotalCapacity"); - registerGauge(jetty.getCoreContainer().getMetricManager(), "solr.jvm", "os.cpuLoad"); - registerGauge(jetty.getCoreContainer().getMetricManager(), "solr.jvm", "os.freeMemorySize"); - registerGauge(jetty.getCoreContainer().getMetricManager(), "solr.jvm", "os.totalMemorySize"); + var cc = solrClientTestRule.getCoreContainer(); + cc.waitForLoadingCoresToFinish(30000); + + SolrMetricManager manager = cc.getMetricManager(); + Counter c = manager.counter(null, "solr.core.collection1", "QUERY./dummy/metrics.requests"); + c.inc(10); + c = manager.counter(null, "solr.node", "ADMIN./dummy/metrics.requests"); + c.inc(20); + Meter m = manager.meter(null, "solr.jetty", "dummyMetrics.2xx-responses"); + m.mark(30); + registerGauge(manager, "solr.jvm", "gc.dummyMetrics.count"); } @Test - public void testPrometheusOutput() throws Exception { + public void testPrometheusStructureOutput() throws Exception { ModifiableSolrParams params = new ModifiableSolrParams(); params.set("qt", "/admin/metrics"); params.set("wt", "prometheus"); QueryRequest req = new QueryRequest(params); req.setResponseParser(new NoOpResponseParser("prometheus")); - try (SolrClient adminClient = getHttpSolrClient(jetty.getBaseUrl().toString()); ) { + + try (SolrClient adminClient = getHttpSolrClient(solrClientTestRule.getBaseUrl())) { + NamedList res = adminClient.request(req); + assertNotNull("null response from server", res); + String output = (String) res.get("response"); + List filteredResponse = + output.lines().filter(line -> !line.startsWith("#")).collect(Collectors.toList()); + filteredResponse.forEach( + (actualMetric) -> { + String actualValue = actualMetric.substring(actualMetric.lastIndexOf("} ") + 1); + assertTrue( + "All metrics should start with 'solr_metrics_'", + actualMetric.startsWith("solr_metrics_")); + try { + Float.parseFloat(actualValue); + } catch (NumberFormatException e) { + log.warn("Prometheus value not a parsable float"); + assertTrue(VALID_PROMETHEUS_VALUES.contains(actualValue)); + } + }); + } + } + + public void testPrometheusDummyOutput() throws Exception { + String expectedCore = + "solr_metrics_core_requests_total{category=\"QUERY\",core=\"collection1\",handler=\"/dummy/metrics\",type=\"requests\"} 10.0"; + String expectedNode = + "solr_metrics_node_requests_total{category=\"ADMIN\",handler=\"/dummy/metrics\",type=\"requests\"} 20.0"; + String expectedJetty = "solr_metrics_jetty_response_total{status=\"2xx\"} 30.0"; + String expectedJvm = "solr_metrics_jvm_gc{item=\"dummyMetrics\"} 0.0"; + + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set("qt", "/admin/metrics"); + params.set("wt", "prometheus"); + QueryRequest req = new QueryRequest(params); + req.setResponseParser(new NoOpResponseParser("prometheus")); + + try (SolrClient adminClient = getHttpSolrClient(solrClientTestRule.getBaseUrl())) { NamedList res = adminClient.request(req); assertNotNull("null response from server", res); - String actual = (String) res.get("response"); - String expectedOutput = - Files.readString(getFile("prometheus/solr-prometheus-output.txt").toPath()); - // Expression to strip out ending numeric values and JVM item tag as we only want to test for - // Prometheus metric names - actual = - actual.replaceAll( - "(?<=}).*|(?<=solr_metrics_jvm_gc\\{)(.*)(?=})|(?<=solr_metrics_jvm_gc_seconds\\{)(.*)(?=})", - ""); - assertEquals(expectedOutput, actual); + String output = (String) res.get("response"); + assertEquals( + expectedCore, + output + .lines() + .filter(line -> line.contains(expectedCore)) + .collect(Collectors.toList()) + .get(0)); + assertEquals( + expectedNode, + output + .lines() + .filter(line -> line.contains(expectedNode)) + .collect(Collectors.toList()) + .get(0)); + assertEquals( + expectedJetty, + output + .lines() + .filter(line -> line.contains(expectedJetty)) + .collect(Collectors.toList()) + .get(0)); + assertEquals( + expectedJvm, + output + .lines() + .filter(line -> line.contains(expectedJvm)) + .collect(Collectors.toList()) + .get(0)); } } From 70a7bd7994088140a11e8ccb640be573f1cce13e Mon Sep 17 00:00:00 2001 From: David Smiley Date: Thu, 1 Aug 2024 01:23:52 -0400 Subject: [PATCH 11/38] CHANGES.txt 9.8.0 --- solr/CHANGES.txt | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 883ee4cd5d5..7e3bf9c1bbd 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -100,6 +100,31 @@ Other Changes * SOLR-17279: Introduce SecurityJson.java file to Test Framework to consolidate setting up authentication in tests. (Rudy Seitz via Eric Pugh) +================== 9.8.0 ================== +New Features +--------------------- +(No changes) + +Improvements +--------------------- +(No changes) + +Optimizations +--------------------- +(No changes) + +Bug Fixes +--------------------- +(No changes) + +Dependency Upgrades +--------------------- +(No changes) + +Other Changes +--------------------- +(No changes) + ================== 9.7.0 ================== New Features --------------------- From 88fa6208c0f2b8cf4b936295940bbe4b39cb778c Mon Sep 17 00:00:00 2001 From: aparnasuresh85 Date: Thu, 1 Aug 2024 01:48:22 -0400 Subject: [PATCH 12/38] SOLR-14985: CloudSolrClient with Solr URLs wasn't caching state (#2571) Solrj CloudSolrClient with Solr URLs had serious performance regressions (since the beginning?) in which its collection state cache was not being used, resulting in many extra requests to Solr for cluster information. (Aparna Suresh, shalin, David Smiley) Co-authored-by: Shalin Shekhar Mangar --- solr/CHANGES.txt | 4 +- .../cloud/DelegatingClusterStateProvider.java | 9 +++ .../client/solrj/impl/CloudSolrClient.java | 21 +++--- .../solrj/impl/ClusterStateProvider.java | 6 +- .../solrj/impl/CloudHttp2SolrClientTest.java | 65 +++++++++++++++++++ 5 files changed, 88 insertions(+), 17 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 7e3bf9c1bbd..7667eb9a34d 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -111,7 +111,9 @@ Improvements Optimizations --------------------- -(No changes) +* SOLR-14985: Solrj CloudSolrClient with Solr URLs had serious performance regressions (since the + beginning?) in which its collection state cache was not being used, resulting in many extra + requests to Solr for cluster information. (Aparna Suresh, shalin, David Smiley) Bug Fixes --------------------- diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingClusterStateProvider.java index b8e7322828f..4177f0a017a 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingClusterStateProvider.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingClusterStateProvider.java @@ -78,6 +78,15 @@ public String resolveSimpleAlias(String alias) throws IllegalArgumentException { } } + @Override + public Object getClusterProperty(String propertyName) { + if (delegate != null) { + return delegate.getClusterProperty(propertyName); + } else { + return null; + } + } + @Override public ClusterState getClusterState() { if (delegate != null) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java index 223fef6ab23..3ac950d0417 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java @@ -362,8 +362,9 @@ private NamedList directUpdate(AbstractUpdateRequest request, String col // Check to see if the collection is an alias. Updates to multi-collection aliases are ok as // long as they are routed aliases - List aliasedCollections = getClusterStateProvider().resolveAlias(collection); - if (getClusterStateProvider().isRoutedAlias(collection) || aliasedCollections.size() == 1) { + List aliasedCollections = + new ArrayList<>(resolveAliases(Collections.singletonList(collection))); + if (aliasedCollections.size() == 1 || getClusterStateProvider().isRoutedAlias(collection)) { collection = aliasedCollections.get(0); // pick 1st (consistent with HttpSolrCall behavior) } else { throw new SolrException( @@ -1149,7 +1150,7 @@ private Set resolveAliases(List inputCollections) { } LinkedHashSet uniqueNames = new LinkedHashSet<>(); // consistent ordering for (String collectionName : inputCollections) { - if (getClusterStateProvider().getState(collectionName) == null) { + if (getDocCollection(collectionName, -1) == null) { // perhaps it's an alias uniqueNames.addAll(getClusterStateProvider().resolveAlias(collectionName)); } else { @@ -1208,15 +1209,6 @@ protected DocCollection getDocCollection(String collection, Integer expectedVers if (expectedVersion <= col.getZNodeVersion() && !cacheEntry.shouldRetry()) return col; } - ClusterState.CollectionRef ref = getCollectionRef(collection); - if (ref == null) { - // no such collection exists - return null; - } - if (!ref.isLazilyLoaded()) { - // it is readily available just return it - return ref.get(); - } Object[] locks = this.locks; int lockId = Math.abs(Hash.murmurhash3_x86_32(collection, 0, collection.length(), 0) % locks.length); @@ -1228,6 +1220,11 @@ protected DocCollection getDocCollection(String collection, Integer expectedVers if (col != null) { if (expectedVersion <= col.getZNodeVersion() && !cacheEntry.shouldRetry()) return col; } + ClusterState.CollectionRef ref = getCollectionRef(collection); + if (ref == null) { + // no such collection exists + return null; + } // We are going to fetch a new version // we MUST try to get a new version DocCollection fetchedCol = ref.get(); // this is a call to ZK diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java index 81bb885c38b..271afca7461 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java @@ -108,15 +108,13 @@ default DocCollection getCollection(String name) throws IOException { /** Obtain a cluster property, or the default value if it doesn't exist. */ default T getClusterProperty(String key, T defaultValue) { @SuppressWarnings({"unchecked"}) - T value = (T) getClusterProperties().get(key); + T value = (T) getClusterProperty(key); if (value == null) return defaultValue; return value; } /** Obtain a cluster property, or null if it doesn't exist. */ - default Object getClusterProperty(String propertyName) { - return getClusterProperties().get(propertyName); - } + Object getClusterProperty(String propertyName); /** Get the collection-specific policy */ String getPolicyNameByCollection(String coll); diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java index 474ebb8a177..6f9cb774beb 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java @@ -35,6 +35,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.regex.Pattern; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.lucene.tests.util.TestUtil; import org.apache.solr.client.solrj.SolrClient; @@ -70,6 +71,9 @@ import org.apache.solr.handler.admin.CollectionsHandler; import org.apache.solr.handler.admin.ConfigSetsHandler; import org.apache.solr.handler.admin.CoreAdminHandler; +import org.apache.solr.servlet.HttpSolrCall; +import org.apache.solr.util.LogLevel; +import org.apache.solr.util.LogListener; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -89,6 +93,15 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase { private static CloudSolrClient httpBasedCloudSolrClient = null; private static CloudSolrClient zkBasedCloudSolrClient = null; + private static final Pattern PATTERN_WITH_COLLECTION = + Pattern.compile( + "path=/admin/collections.*params=\\{[^}]*action=CLUSTERSTATUS" + + "[^}]*collection=[^&}]+[^}]*\\}"); + private static final Pattern PATTERN_WITHOUT_COLLECTION = + Pattern.compile( + "path=/admin/collections.*params=\\{[^}]*action=CLUSTERSTATUS" + + "(?![^}]*collection=)[^}]*\\}"); + @BeforeClass public static void setupCluster() throws Exception { System.setProperty("metricsEnabled", "true"); @@ -114,6 +127,7 @@ public static void setupCluster() throws Exception { @AfterClass public static void tearDownAfterClass() throws Exception { + if (httpBasedCloudSolrClient != null) { try { httpBasedCloudSolrClient.close(); @@ -246,6 +260,57 @@ public void testAliasHandling() throws Exception { 2, client.query(null, paramsWithMixedCollectionAndAlias).getResults().getNumFound()); } + @Test + @LogLevel("org.apache.solr.servlet.HttpSolrCall=DEBUG") + public void testHttpCspPerf() throws Exception { + + String collectionName = "HTTPCSPTEST"; + CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1) + .process(cluster.getSolrClient()); + cluster.waitForActiveCollection(collectionName, 2, 2); + + try (LogListener entireClusterStateLogs = + LogListener.info(HttpSolrCall.class).regex(PATTERN_WITHOUT_COLLECTION); + LogListener collectionClusterStateLogs = + LogListener.info(HttpSolrCall.class).regex(PATTERN_WITH_COLLECTION); + LogListener adminRequestLogs = LogListener.info(HttpSolrCall.class).substring("[admin]"); + CloudSolrClient solrClient = createHttpCSPBasedCloudSolrClient(); ) { + SolrInputDocument doc = new SolrInputDocument("id", "1", "title_s", "my doc"); + solrClient.add(collectionName, doc); + solrClient.commit(collectionName); + for (int i = 0; i < 3; i++) { + assertEquals( + 1, solrClient.query(collectionName, params("q", "*:*")).getResults().getNumFound()); + } + + // 1 call to fetch entire cluster state via BaseHttpCSP.fetchLiveNodes() + // 1 call to fetch CLUSTERSTATUS for collection via getDocCollection() (first collection + // lookup) + assertLogCount(adminRequestLogs, 2); + // 1 call to fetch CLUSTERSTATUS for collection via getDocCollection() (first collection + // lookup) + assertLogCount(collectionClusterStateLogs, 1); + // 1 call to fetch entire cluster state from HttpCSP.fetchLiveNodes() + assertLogCount(entireClusterStateLogs, 1); + } + } + + private CloudSolrClient createHttpCSPBasedCloudSolrClient() { + final List solrUrls = new ArrayList<>(); + solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString()); + return new CloudHttp2SolrClient.Builder(solrUrls).build(); + } + + private void assertLogCount(LogListener logListener, int expectedCount) { + int logCount = logListener.getCount(); + assertEquals(expectedCount, logCount); + if (logCount > 0) { + for (int i = 0; i < logCount; i++) { + logListener.pollMessage(); + } + } + } + @Test public void testRouting() throws Exception { CollectionAdminRequest.createCollection("routing_collection", "conf", 2, 1) From c30e2d01db076ada751c5d9bfaa7e403e77d33d4 Mon Sep 17 00:00:00 2001 From: Christos Malliaridis Date: Fri, 2 Aug 2024 15:27:53 +0300 Subject: [PATCH 13/38] SOLR-16996: Update Solr Exporter for Prometheus cli to use commons-cli (#2602) * Migrate SolrExporter to use commons-cli and remove argparse4j * now trigger bats testing when anything changes in the exporter code base to provide better testing --------- Co-authored-by: Eric Pugh --- .github/workflows/bin-solr-test.yml | 2 +- solr/prometheus-exporter/build.gradle | 2 +- .../prometheus/exporter/SolrExporter.java | 340 +++++++++--------- versions.lock | 1 - versions.props | 1 - 5 files changed, 178 insertions(+), 168 deletions(-) diff --git a/.github/workflows/bin-solr-test.yml b/.github/workflows/bin-solr-test.yml index 1f07d4367a6..0754fcd46bd 100644 --- a/.github/workflows/bin-solr-test.yml +++ b/.github/workflows/bin-solr-test.yml @@ -10,6 +10,7 @@ on: - 'solr/bin/**' - 'solr/packaging/**' - 'solr/core/src/java/org/apache/solr/cli/**' + - 'solr/prometheus-exporter/**' jobs: test: @@ -47,4 +48,3 @@ jobs: with: name: logs path: solr/packaging/build/test-output - diff --git a/solr/prometheus-exporter/build.gradle b/solr/prometheus-exporter/build.gradle index 66beb70aed7..0739bdf1c53 100644 --- a/solr/prometheus-exporter/build.gradle +++ b/solr/prometheus-exporter/build.gradle @@ -43,7 +43,7 @@ dependencies { exclude group: "com.fasterxml.jackson.core", module: "jackson-databind" }) implementation ('com.fasterxml.jackson.core:jackson-databind') - implementation ('net.sourceforge.argparse4j:argparse4j') + implementation 'commons-cli:commons-cli' implementation ('com.github.ben-manes.caffeine:caffeine') { transitive = false } implementation 'org.slf4j:slf4j-api' implementation 'commons-codec:commons-codec' diff --git a/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java b/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java index 73aec2c567f..b7918b90901 100644 --- a/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java +++ b/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java @@ -23,10 +23,14 @@ import java.net.InetSocketAddress; import java.nio.file.Paths; import java.util.concurrent.ExecutorService; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.DeprecatedAttributes; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; import org.apache.commons.codec.digest.DigestUtils; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.IOUtils; @@ -44,72 +48,14 @@ public class SolrExporter { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final String[] ARG_PORT_FLAGS = {"-p", "--port"}; - private static final String ARG_PORT_METAVAR = "PORT"; - private static final String ARG_PORT_DEST = "port"; - private static final int ARG_PORT_DEFAULT = 8989; - private static final String ARG_PORT_HELP = - "Specify the solr-exporter HTTP listen port; default is " + ARG_PORT_DEFAULT + "."; - - private static final String[] ARG_BASE_URL_FLAGS = {"-b", "--baseurl"}; - private static final String ARG_BASE_URL_METAVAR = "BASE_URL"; - private static final String ARG_BASE_URL_DEST = "baseUrl"; - private static final String ARG_BASE_URL_DEFAULT = "http://localhost:8983/solr"; - private static final String ARG_BASE_URL_HELP = - "Specify the Solr base URL when connecting to Solr in standalone mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'http://localhost:8983/solr'."; - - private static final String[] ARG_ZK_HOST_FLAGS = {"-z", "--zk-host"}; - private static final String ARG_ZK_HOST_METAVAR = "ZK_HOST"; - private static final String ARG_ZK_HOST_DEST = "zkHost"; - private static final String ARG_ZK_HOST_DEFAULT = ""; - private static final String ARG_ZK_HOST_HELP = - "Specify the ZooKeeper connection string when connecting to Solr in SolrCloud mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'localhost:2181/solr'."; - - private static final String[] ARG_CLUSTER_ID_FLAGS = {"-i", "--cluster-id"}; - private static final String ARG_CLUSTER_ID_METAVAR = "CLUSTER_ID"; - private static final String ARG_CLUSTER_ID_DEST = "clusterId"; - private static final String ARG_CLUSTER_ID_DEFAULT = ""; - private static final String ARG_CLUSTER_ID_HELP = - "Specify a unique identifier for the cluster, which can be used to select between multiple clusters in Grafana. By default this ID will be equal to a hash of the -b or -z argument"; - - private static final String[] ARG_CONFIG_FLAGS = {"-f", "--config-file"}; - private static final String ARG_CONFIG_METAVAR = "CONFIG"; - private static final String ARG_CONFIG_DEST = "configFile"; - private static final String ARG_CONFIG_DEFAULT = "solr-exporter-config.xml"; - private static final String ARG_CONFIG_HELP = - "Specify the configuration file; the default is " + ARG_CONFIG_DEFAULT + "."; - - private static final String[] ARG_SCRAPE_INTERVAL_FLAGS = {"-s", "--scrape-interval"}; - private static final String ARG_SCRAPE_INTERVAL_METAVAR = "SCRAPE_INTERVAL"; - private static final String ARG_SCRAPE_INTERVAL_DEST = "scrapeInterval"; - private static final int ARG_SCRAPE_INTERVAL_DEFAULT = 60; - private static final String ARG_SCRAPE_INTERVAL_HELP = - "Specify the delay between scraping Solr metrics; the default is " - + ARG_SCRAPE_INTERVAL_DEFAULT - + " seconds."; - - private static final String[] ARG_NUM_THREADS_FLAGS = {"-n", "--num-threads"}; - private static final String ARG_NUM_THREADS_METAVAR = "NUM_THREADS"; - private static final String ARG_NUM_THREADS_DEST = "numThreads"; - private static final Integer ARG_NUM_THREADS_DEFAULT = 1; - private static final String ARG_NUM_THREADS_HELP = - "Specify the number of threads. solr-exporter creates a thread pools for request to Solr. If you need to improve request latency via solr-exporter, you can increase the number of threads; the default is " - + ARG_NUM_THREADS_DEFAULT - + "."; - - private static final String[] ARG_CREDENTIALS_FLAGS = {"-u", "--credentials"}; - private static final String ARG_CREDENTIALS_METAVAR = "CREDENTIALS"; - private static final String ARG_CREDENTIALS_DEST = "credentials"; - private static final String ARG_CREDENTIALS_DEFAULT = ""; - private static final String ARG_CREDENTIALS_HELP = - "Specify the credentials in the format username:password. Example: --credentials solr:SolrRocks"; - - private static final String[] ARG_SSL_FLAGS = {"-ssl", "--ssl-enabled"}; - private static final String ARG_SSL_METAVAR = "SSL_ENABLED"; - private static final String ARG_SSL_DEST = "ssl_enabled"; - private static final boolean ARG_SSL_DEFAULT = false; - private static final String ARG_SSL_HELP = - "Enable TLS connection to Solr. Expects following env variables: SOLR_SSL_KEY_STORE, SOLR_SSL_KEY_STORE_PASSWORD, SOLR_SSL_TRUST_STORE, SOLR_SSL_TRUST_STORE_PASSWORD. Example: --ssl-enabled"; + private static final int DEFAULT_PORT = 8989; + private static final String DEFAULT_BASE_URL = "http://localhost:8983/solr"; + private static final String DEFAULT_ZK_HOST = ""; + private static final String DEFAULT_CLUSTER_ID = ""; + private static final String DEFAULT_CONFIG = "solr-exporter-config.xml"; + private static final int DEFAULT_SCRAPE_INTERVAL = 60; + private static final Integer DEFAULT_NUM_THREADS = 1; + private static final String DEFAULT_CREDENTIALS = ""; public static final CollectorRegistry defaultRegistry = new CollectorRegistry(); @@ -196,116 +142,182 @@ private SolrScraper createScraper( } public static void main(String[] args) { - ArgumentParser parser = - ArgumentParsers.newFor(SolrExporter.class.getSimpleName()) - .build() - .description("Prometheus exporter for Apache Solr."); - - parser - .addArgument(ARG_PORT_FLAGS) - .metavar(ARG_PORT_METAVAR) - .dest(ARG_PORT_DEST) - .type(Integer.class) - .setDefault(ARG_PORT_DEFAULT) - .help(ARG_PORT_HELP); - - parser - .addArgument(ARG_BASE_URL_FLAGS) - .metavar(ARG_BASE_URL_METAVAR) - .dest(ARG_BASE_URL_DEST) - .type(String.class) - .setDefault(ARG_BASE_URL_DEFAULT) - .help(ARG_BASE_URL_HELP); - - parser - .addArgument(ARG_ZK_HOST_FLAGS) - .metavar(ARG_ZK_HOST_METAVAR) - .dest(ARG_ZK_HOST_DEST) - .type(String.class) - .setDefault(ARG_ZK_HOST_DEFAULT) - .help(ARG_ZK_HOST_HELP); - - parser - .addArgument(ARG_CONFIG_FLAGS) - .metavar(ARG_CONFIG_METAVAR) - .dest(ARG_CONFIG_DEST) - .type(String.class) - .setDefault(ARG_CONFIG_DEFAULT) - .help(ARG_CONFIG_HELP); - - parser - .addArgument(ARG_SCRAPE_INTERVAL_FLAGS) - .metavar(ARG_SCRAPE_INTERVAL_METAVAR) - .dest(ARG_SCRAPE_INTERVAL_DEST) - .type(Integer.class) - .setDefault(ARG_SCRAPE_INTERVAL_DEFAULT) - .help(ARG_SCRAPE_INTERVAL_HELP); - - parser - .addArgument(ARG_NUM_THREADS_FLAGS) - .metavar(ARG_NUM_THREADS_METAVAR) - .dest(ARG_NUM_THREADS_DEST) - .type(Integer.class) - .setDefault(ARG_NUM_THREADS_DEFAULT) - .help(ARG_NUM_THREADS_HELP); - - parser - .addArgument(ARG_CLUSTER_ID_FLAGS) - .metavar(ARG_CLUSTER_ID_METAVAR) - .dest(ARG_CLUSTER_ID_DEST) - .type(String.class) - .setDefault(ARG_CLUSTER_ID_DEFAULT) - .help(ARG_CLUSTER_ID_HELP); - - parser - .addArgument(ARG_CREDENTIALS_FLAGS) - .metavar(ARG_CREDENTIALS_METAVAR) - .dest(ARG_CREDENTIALS_DEST) - .type(String.class) - .setDefault(ARG_CREDENTIALS_DEFAULT) - .help(ARG_CREDENTIALS_HELP); - - parser - .addArgument(ARG_SSL_FLAGS) - .metavar(ARG_SSL_METAVAR) - .dest(ARG_SSL_DEST) - .type(Boolean.class) - .setDefault(ARG_SSL_DEFAULT) - .help(ARG_SSL_HELP); + Options mainOptions = new Options(); + Options deprecatedOptions = new Options(); + + Option baseUrlOption = + Option.builder("b") + .longOpt("base-url") + .hasArg() + .argName("BASE_URL") + .type(String.class) + .desc( + "Specify the Solr base URL when connecting to Solr in standalone mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'http://localhost:8983/solr'.") + .build(); + mainOptions.addOption(baseUrlOption); + + Option baseUrlDepOption = + Option.builder() + .longOpt("baseUrl") + .hasArg() + .argName("BASE_URL") + .type(String.class) + .deprecated( + DeprecatedAttributes.builder() + .setForRemoval(true) + .setSince("9.7") + .setDescription("Use --base-url instead") + .get()) + .desc( + "Specify the Solr base URL when connecting to Solr in standalone mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'http://localhost:8983/solr'.") + .build(); + deprecatedOptions.addOption(baseUrlDepOption); + + Option configOption = + Option.builder("f") + .longOpt("config-file") + .hasArg() + .argName("CONFIG") + .type(String.class) + .desc("Specify the configuration file; the default is " + DEFAULT_CONFIG + ".") + .build(); + mainOptions.addOption(configOption); + + Option helpOption = + Option.builder("h").longOpt("help").desc("Prints this help message.").build(); + mainOptions.addOption(helpOption); + + Option clusterIdOption = + Option.builder("i") + .longOpt("cluster-id") + .hasArg() + .argName("CLUSTER_ID") + .type(String.class) + .desc( + "Specify a unique identifier for the cluster, which can be used to select between multiple clusters in Grafana. By default this ID will be equal to a hash of the -b or -z argument") + .build(); + mainOptions.addOption(clusterIdOption); + + Option numThreadsOption = + Option.builder("n") + .longOpt("num-threads") + .hasArg() + .argName("NUM_THREADS") + .type(Integer.class) + .desc( + "Specify the number of threads. solr-exporter creates a thread pools for request to Solr. If you need to improve request latency via solr-exporter, you can increase the number of threads; the default is " + + DEFAULT_NUM_THREADS + + ".") + .build(); + mainOptions.addOption(numThreadsOption); + + Option portOption = + Option.builder("p") + .longOpt("port") + .hasArg() + .argName("PORT") + .type(Integer.class) + .desc("Specify the solr-exporter HTTP listen port; default is " + DEFAULT_PORT + ".") + .build(); + mainOptions.addOption(portOption); + + Option scrapeIntervalOption = + Option.builder("s") + .longOpt("scrape-interval") + .hasArg() + .argName("SCRAPE_INTERVAL") + .type(Integer.class) + .desc( + "Specify the delay between scraping Solr metrics; the default is " + + DEFAULT_SCRAPE_INTERVAL + + " seconds.") + .build(); + mainOptions.addOption(scrapeIntervalOption); + + Option sslOption = + Option.builder("ssl") + .longOpt("ssl-enabled") + .type(Boolean.class) + .desc( + "Enable TLS connection to Solr. Expects following env variables: SOLR_SSL_KEY_STORE, SOLR_SSL_KEY_STORE_PASSWORD, SOLR_SSL_TRUST_STORE, SOLR_SSL_TRUST_STORE_PASSWORD. Example: --ssl-enabled") + .build(); + mainOptions.addOption(sslOption); + + Option credentialsOption = + Option.builder("u") + .longOpt("credentials") + .hasArg() + .argName("CREDENTIALS") + .type(String.class) + .desc( + "Specify the credentials in the format username:password. Example: --credentials solr:SolrRocks") + .build(); + mainOptions.addOption(credentialsOption); + + Option zkHostOption = + Option.builder("z") + .longOpt("zk-host") + .hasArg() + .argName("ZK_HOST") + .type(String.class) + .desc( + "Specify the ZooKeeper connection string when connecting to Solr in SolrCloud mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'localhost:2181/solr'.") + .build(); + mainOptions.addOption(zkHostOption); + + Options options = new Options(); + options.addOptions(mainOptions); + options.addOptions(deprecatedOptions); try { - Namespace res = parser.parseArgs(args); + CommandLineParser parser = new DefaultParser(); + CommandLine commandLine = parser.parse(options, args); + + if (commandLine.hasOption(helpOption)) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp( + "bin/solr-exporter", "Prometheus exporter for Apache Solr.", mainOptions, null, true); + return; + } SolrScrapeConfiguration scrapeConfiguration = null; String defaultClusterId = ""; - if (!res.getString(ARG_ZK_HOST_DEST).isEmpty()) { - defaultClusterId = makeShortHash(res.getString(ARG_ZK_HOST_DEST)); - scrapeConfiguration = SolrScrapeConfiguration.solrCloud(res.getString(ARG_ZK_HOST_DEST)); - } else if (!res.getString(ARG_BASE_URL_DEST).isEmpty()) { - defaultClusterId = makeShortHash(res.getString(ARG_BASE_URL_DEST)); - scrapeConfiguration = SolrScrapeConfiguration.standalone(res.getString(ARG_BASE_URL_DEST)); + if (commandLine.hasOption(zkHostOption)) { + String zkHost = commandLine.getOptionValue(zkHostOption, DEFAULT_ZK_HOST); + defaultClusterId = makeShortHash(zkHost); + scrapeConfiguration = SolrScrapeConfiguration.solrCloud(zkHost); + } else if (commandLine.hasOption(baseUrlOption) || commandLine.hasOption(baseUrlDepOption)) { + String baseUrl = + commandLine.hasOption(baseUrlOption) + ? commandLine.getOptionValue(baseUrlOption) + : commandLine.getOptionValue(baseUrlDepOption, DEFAULT_BASE_URL); + defaultClusterId = makeShortHash(baseUrl); + scrapeConfiguration = SolrScrapeConfiguration.standalone(baseUrl); } if (scrapeConfiguration == null) { - log.error("Must provide either {} or {}", ARG_BASE_URL_FLAGS, ARG_ZK_HOST_FLAGS); + log.error( + "Must provide either --{} or --{}", + baseUrlOption.getLongOpt(), + zkHostOption.getLongOpt()); } - int port = res.getInt(ARG_PORT_DEST); - String clusterId = res.getString(ARG_CLUSTER_ID_DEST); + int port = commandLine.getParsedOptionValue(portOption, DEFAULT_PORT); + String clusterId = commandLine.getOptionValue(clusterIdOption, DEFAULT_CLUSTER_ID); if (StrUtils.isNullOrEmpty(clusterId)) { clusterId = defaultClusterId; } - if (!res.getString(ARG_CREDENTIALS_DEST).isEmpty()) { - String credentials = res.getString(ARG_CREDENTIALS_DEST); + if (commandLine.hasOption(credentialsOption)) { + String credentials = commandLine.getOptionValue(credentialsOption, DEFAULT_CREDENTIALS); if (credentials.indexOf(':') > 0) { String[] credentialsArray = credentials.split(":", 2); scrapeConfiguration.withBasicAuthCredentials(credentialsArray[0], credentialsArray[1]); } } - if (Boolean.TRUE.equals(res.getBoolean(ARG_SSL_DEST))) { + if (commandLine.hasOption(sslOption)) { log.info("SSL ENABLED"); scrapeConfiguration.withSslConfiguration( @@ -318,10 +330,10 @@ public static void main(String[] args) { SolrExporter solrExporter = new SolrExporter( port, - res.getInt(ARG_NUM_THREADS_DEST), - res.getInt(ARG_SCRAPE_INTERVAL_DEST), + commandLine.getParsedOptionValue(numThreadsOption, DEFAULT_NUM_THREADS), + commandLine.getParsedOptionValue(scrapeIntervalOption, DEFAULT_SCRAPE_INTERVAL), scrapeConfiguration, - loadMetricsConfiguration(res.getString(ARG_CONFIG_DEST)), + loadMetricsConfiguration(commandLine.getOptionValue(configOption, DEFAULT_CONFIG)), clusterId); log.info("Starting Solr Prometheus Exporting on port {}", port); @@ -332,8 +344,8 @@ public static void main(String[] args) { scrapeConfiguration); } catch (IOException e) { log.error("Failed to start Solr Prometheus Exporter: ", e); - } catch (ArgumentParserException e) { - parser.handleError(e); + } catch (ParseException e) { + log.error("Failed to parse command line arguments: ", e); } } diff --git a/versions.lock b/versions.lock index 4e612680d9f..1e2896b7f22 100644 --- a/versions.lock +++ b/versions.lock @@ -165,7 +165,6 @@ net.arnx:jsonic:1.2.7 (1 constraints: d00b47eb) net.java.dev.jna:jna:5.12.1 (1 constraints: 900c8e0e) net.sf.ehcache:ehcache-core:2.6.2 (1 constraints: 2706f94d) net.sf.jopt-simple:jopt-simple:5.0.4 (1 constraints: be0ad6cc) -net.sourceforge.argparse4j:argparse4j:0.9.0 (1 constraints: 0b050636) net.thisptr:jackson-jq:0.0.13 (1 constraints: 3605223b) org.antlr:antlr4-runtime:4.11.1 (1 constraints: f70fbd96) org.apache.calcite:calcite-core:1.37.0 (1 constraints: 3d05463b) diff --git a/versions.props b/versions.props index 844bf553fdf..35e2b13390b 100644 --- a/versions.props +++ b/versions.props @@ -29,7 +29,6 @@ io.prometheus:*=0.16.0 io.swagger.core.v3:*=2.2.22 jakarta.ws.rs:jakarta.ws.rs-api=3.1.0 junit:junit=4.13.2 -net.sourceforge.argparse4j:argparse4j=0.9.0 net.thisptr:jackson-jq=0.0.13 no.nav.security:mock-oauth2-server=0.5.10 org.apache.calcite.avatica:avatica-core=1.25.0 From 74bfadfc5519b4fcc052009521bd294f30133035 Mon Sep 17 00:00:00 2001 From: Eric Pugh Date: Fri, 2 Aug 2024 08:30:10 -0400 Subject: [PATCH 14/38] add missing change entry --- solr/CHANGES.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 7667eb9a34d..8787dd60653 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -279,6 +279,9 @@ Other Changes * SOLR-17322: Once again allow rank queries to use custom TopDocsCollectors that operate on types that extend ScoreDocs (covariant generic types) broken in Solr 9.0. (Stephen Woods via Christine Poerschke) +* SOLR-16996: Update Solr Exporter for Prometheus cli to use commons-cli instead of argparse4j. (Christos Malliaridis via Eric Pugh) + + ================== 9.6.1 ================== Bug Fixes --------------------- From b44bdc198d835ad7ebc07c5b4b0cb2e9c6c8ab74 Mon Sep 17 00:00:00 2001 From: Houston Putman Date: Fri, 2 Aug 2024 12:29:18 -0400 Subject: [PATCH 15/38] SOLR-10808: Enable docValues by default on specific fields (#2606) Fields: - PrimitiveField (Numbers/Dates/String/UUID/Enum/Bool) - Without DenseVectors - SortableTextField - SortableBinaryField - ICUCollationField - CollationField - LatLonPointSpacialField --- solr/CHANGES.txt | 15 +++++++------ .../org/apache/solr/schema/BinaryField.java | 5 ----- .../apache/solr/schema/CollationField.java | 2 +- .../apache/solr/schema/DenseVectorField.java | 2 +- .../org/apache/solr/schema/FieldType.java | 22 ++++++++++--------- .../solr/schema/LatLonPointSpatialField.java | 2 +- .../org/apache/solr/schema/PointType.java | 5 ----- .../solr/schema/PrimitiveFieldType.java | 2 +- .../apache/solr/schema/SortableTextField.java | 2 +- .../collection1/conf/schema-binaryfield.xml | 2 +- .../apache/solr/schema/ICUCollationField.java | 2 +- .../indexing-guide/pages/docvalues.adoc | 16 +++++++++++++- ...field-type-definitions-and-properties.adoc | 2 +- .../query-guide/pages/spatial-search.adoc | 2 +- .../pages/major-changes-in-solr-9.adoc | 7 +++--- .../solr/schema/SortableBinaryField.java | 2 +- 16 files changed, 49 insertions(+), 41 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 8787dd60653..99533e5f97c 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -153,6 +153,14 @@ New Features Improvements --------------------- +* SOLR-10808, SOLR-12963: The Solr schema version has been increased to 1.7. + Starting in schema version 1.7, most fields/fieldTypes that support docValues will have them enabled by default. + These field types include primitive (Numeric, Date, Bool, String, Enum, UUID), sorting (SortableTextField, SortableBinaryField, CollationField, ICUCollationField) and LatLonPointSpacialField. + This behavior can be reverted by setting the 'docValues' parameter for a field or a field type to false, the default for schema versions 1.6 and below. + Also in schema version 1.7, all fields/fieldTypes will be unable to be uninverted by default. + This behavior can be reverted by setting the 'uninvertible' parameter for a field or a field type to true, the default for schema versions 1.6 and below. + (Houston Putman, hossman) + * SOLR-17137: Enable Prometheus exporter to communicate with SSL protected Solr. (Eivind Bergstøl via Eric Pugh) * SOLR-16921: use -solrUrl to derive the zk host connection for bin/solr zk subcommands (Eric Pugh) @@ -183,13 +191,6 @@ Improvements Statuses are now removed 5 minutes after the read of a completed/failed status. Helps collection async backup/restore and other operations scale to 100+ shards. (Pierre Salagnac, David Smiley) -* SOLR-10808, SOLR-12963: The Solr schema version has been increased to 1.7. - Starting in schema version 1.7, all fields/fieldTypes that support docValues will have them enabled by default. - This behavior can be reverted by setting the 'docValues' parameter for a field or a field type to false, the default for schema versions 1.6 and below. - Also in schema version 1.7, all fields/fieldTypes will be unable to be uninverted by default. - This behavior can be reverted by setting the 'uninvertible' parameter for a field or a field type to true, the default for schema versions 1.6 and below. - (Houston Putman, hossman) - * SOLR-10808 : The Solr schema version has been increased to 1.7. Since schema version 1.7, all fields/fieldTypes that support docValues will have them enabled by default. This behavior can be reverted by setting 'docValues' parameter for a field or a field type to false, the default for schema versions 1.6 and below. (Houston Putman) diff --git a/solr/core/src/java/org/apache/solr/schema/BinaryField.java b/solr/core/src/java/org/apache/solr/schema/BinaryField.java index 37bfc653236..94bc94fd86f 100644 --- a/solr/core/src/java/org/apache/solr/schema/BinaryField.java +++ b/solr/core/src/java/org/apache/solr/schema/BinaryField.java @@ -152,11 +152,6 @@ public List createFields(SchemaField field, Object val) { protected void checkSupportsDocValues() { // we support DocValues } - @Override - protected boolean doesTypeSupportDocValues() { - return true; - } - @Override public Object toNativeType(Object val) { if (val instanceof byte[]) { diff --git a/solr/core/src/java/org/apache/solr/schema/CollationField.java b/solr/core/src/java/org/apache/solr/schema/CollationField.java index 739bae5f211..91f67d459e7 100644 --- a/solr/core/src/java/org/apache/solr/schema/CollationField.java +++ b/solr/core/src/java/org/apache/solr/schema/CollationField.java @@ -246,7 +246,7 @@ protected void checkSupportsDocValues() { // we support DocValues } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } diff --git a/solr/core/src/java/org/apache/solr/schema/DenseVectorField.java b/solr/core/src/java/org/apache/solr/schema/DenseVectorField.java index d83462235b0..4d528361dd4 100644 --- a/solr/core/src/java/org/apache/solr/schema/DenseVectorField.java +++ b/solr/core/src/java/org/apache/solr/schema/DenseVectorField.java @@ -179,7 +179,7 @@ public VectorEncoding getVectorEncoding() { } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return false; } diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java index 9431691b1de..cb60b1f332a 100644 --- a/solr/core/src/java/org/apache/solr/schema/FieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java @@ -188,7 +188,7 @@ protected void setArgs(IndexSchema schema, Map args) { args.remove("compressThreshold"); } if (schemaVersion >= 1.6f) properties |= USE_DOCVALUES_AS_STORED; - if (schemaVersion >= 1.7f && doesTypeSupportDocValues()) properties |= DOC_VALUES; + if (schemaVersion >= 1.7f && enableDocValuesByDefault()) properties |= DOC_VALUES; if (schemaVersion < 1.7f) properties |= UNINVERTIBLE; @@ -1161,15 +1161,17 @@ protected void checkSupportsDocValues() { ErrorCode.SERVER_ERROR, "Field type " + this + " does not support doc values"); } - /** Returns whether this field type supports docValues. By default none do. */ - protected boolean doesTypeSupportDocValues() { - try { - // TODO: In Solr 10.0 change this such that checkSupportsDocValues() calls this method instead - checkSupportsDocValues(); - return true; - } catch (Exception ignored) { - return false; - } + /** + * Returns whether this field type should enable docValues by default for schemaVersion >= 1.7. + * This should not be enabled for fields that did not have docValues implemented by Solr 9.7, as + * users may have indexed documents without docValues (since they weren't supported). Flipping the + * default docValues values when they upgrade to a new version will break their index + * compatibility. + * + *

New field types can enable this without issue, as long as they support docValues. + */ + protected boolean enableDocValuesByDefault() { + return false; } public static final String TYPE = "type"; diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java index b237b79ad37..5010ba27075 100644 --- a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java +++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java @@ -70,7 +70,7 @@ protected void checkSupportsDocValues() { // we support DocValues } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } diff --git a/solr/core/src/java/org/apache/solr/schema/PointType.java b/solr/core/src/java/org/apache/solr/schema/PointType.java index 2f80fc24722..3d68af9449c 100644 --- a/solr/core/src/java/org/apache/solr/schema/PointType.java +++ b/solr/core/src/java/org/apache/solr/schema/PointType.java @@ -172,11 +172,6 @@ protected void checkSupportsDocValues() { } } - @Override - protected boolean doesTypeSupportDocValues() { - return true; - } - /** * Calculates the range and creates a RangeQuery (bounding box) wrapped in a BooleanQuery (unless * the dimension is 1, one range for every dimension, AND'd together by a Boolean diff --git a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java index 2ce3203ae71..cc3fb3f7ff0 100644 --- a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java @@ -39,7 +39,7 @@ protected void checkSupportsDocValues() { // primitive types support DocValues } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } diff --git a/solr/core/src/java/org/apache/solr/schema/SortableTextField.java b/solr/core/src/java/org/apache/solr/schema/SortableTextField.java index 4811a1dfa59..9bc6a3f7cb9 100644 --- a/solr/core/src/java/org/apache/solr/schema/SortableTextField.java +++ b/solr/core/src/java/org/apache/solr/schema/SortableTextField.java @@ -151,7 +151,7 @@ protected void checkSupportsDocValues() { } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml b/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml index 7d5b3f09051..51c2b26a832 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml @@ -33,7 +33,7 @@ - + id diff --git a/solr/modules/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java b/solr/modules/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java index 87a09c56212..9d72b88ee38 100644 --- a/solr/modules/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java +++ b/solr/modules/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java @@ -316,7 +316,7 @@ protected void checkSupportsDocValues() { // we support DocValues } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } diff --git a/solr/solr-ref-guide/modules/indexing-guide/pages/docvalues.adoc b/solr/solr-ref-guide/modules/indexing-guide/pages/docvalues.adoc index 1cf3edf9b92..34b836def78 100644 --- a/solr/solr-ref-guide/modules/indexing-guide/pages/docvalues.adoc +++ b/solr/solr-ref-guide/modules/indexing-guide/pages/docvalues.adoc @@ -35,7 +35,21 @@ This approach promises to relieve some of the memory requirements of the fieldCa == Enabling DocValues [IMPORTANT] -DocValues are enabled by default for all field types that support them (when using a schemaVersion >= 1.7). +DocValues are enabled by default for most field types that support them *when using a schemaVersion >= 1.7*. + +* Primitive Fields +** `Numeric` (not `DenseVectorField`) +** `Boolean` +** `String` +** `Date` +** `UUID` +** `Enum` +* Sorting Fields +** `CollationField` +** `ICUCollationField` +** `SortableTextField` +** `SortableBinaryField` +* `LatLonPointSpacialField` (not `PointType`, which is different than `PointField`) When using an earlier schemaVersion (\<= 1.6), you only need to enable docValues for a field that you will use it with. As with all schema design, you need to define a field type and then define fields of that type with docValues enabled. diff --git a/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc b/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc index f085682b6cb..ee613d53cf9 100644 --- a/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc +++ b/solr/solr-ref-guide/modules/indexing-guide/pages/field-type-definitions-and-properties.adoc @@ -198,7 +198,7 @@ The table below includes the default value for most `FieldType` implementations |Property |Description |Implicit Default |`indexed` |If `true`, the value of the field can be used in queries to retrieve matching documents. |`true` |`stored` |If `true`, the actual value of the field can be retrieved by queries. |`true` -|`docValues` |If `true`, the value of the field will be put in a column-oriented xref:docvalues.adoc[] structure. |`true` +|`docValues` |If `true`, the value of the field will be put in a column-oriented xref:docvalues.adoc[] structure. |`true` for xref:indexing-guide:docvalues.adoc#enabling-docvalues[_most_ fields] |`sortMissingFirst`, `sortMissingLast` |Control the placement of documents when a sort field is not present. |`false` |`multiValued` |If `true`, indicates that a single document might contain multiple values for this field type. |`false` |`uninvertible` |If `true`, indicates that an `indexed="true" docValues="false"` field can be "un-inverted" at query time to build up large in memory data structure to serve in place of xref:docvalues.adoc[]. |`false` diff --git a/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc b/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc index 868938e31e5..cb714e2f8ac 100644 --- a/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc +++ b/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc @@ -27,7 +27,7 @@ Using spatial search, you can: There are four main field types available for spatial search: -* `LatLonPointSpatialField` +* `LatLonPointSpatialField` (has `docValues` enabled by default) * `PointType` * `SpatialRecursivePrefixTreeFieldType` (RPT for short), including `RptWithGeometrySpatialField`, a derivative * `BBoxField` diff --git a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc index ee1cae69deb..e6413f8cd15 100644 --- a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc +++ b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc @@ -70,8 +70,9 @@ Due to changes in Lucene 9, that isn't possible any more. == Solr 9.7 === SchemaVersion upgraded to 1.7 The default schemaVersion is now 1.7. -With the new schema version, all fields and fieldTypes that support docValues (StrField, *PointField, BoolField, etc.) -will have docValues enabled by default. +With the new schema version, most fields and fieldTypes that support docValues will have docValues enabled by default. +See the xref:indexing-guide:docvalues.adoc#enabling-docvalues[Enabling Doc Values Section] for a complete list. + Also, fields will be unable to be uninverted by default. (`uninvertible=false`) In order to upgrade from schemaVersion 1.6 to 1.7, a re-index of all data will generally be required. @@ -81,7 +82,7 @@ https://issues.apache.org/jira/browse/LUCENE-9334[Lucene does not allow adding d However, if any of the following is true, a re-index is not required: * All applicable fields already have docValues enabled in the original schema -* The schema is updated to explicitly default `docValues="false"` for all fields and fieldTypes that did not have an explicit `docValues` default provided. +* The schema is updated to xref:indexing-guide:docvalues.adoc#disabling-docvalues[explicitly disable docValues] for all fields and fieldTypes that did not have an explicit `docValues` default provided. The `uninvertible=false` default has no impact on the index, so is unrelated to re-indexing concerns. diff --git a/solr/test-framework/src/java/org/apache/solr/schema/SortableBinaryField.java b/solr/test-framework/src/java/org/apache/solr/schema/SortableBinaryField.java index 5c6d31c43ba..28edf725d14 100644 --- a/solr/test-framework/src/java/org/apache/solr/schema/SortableBinaryField.java +++ b/solr/test-framework/src/java/org/apache/solr/schema/SortableBinaryField.java @@ -37,7 +37,7 @@ protected void checkSupportsDocValues() { // we support DocValues } @Override - protected boolean doesTypeSupportDocValues() { + protected boolean enableDocValuesByDefault() { return true; } From 030b0dcd02b0d75b36da0da044bda138c59f047d Mon Sep 17 00:00:00 2001 From: Christos Malliaridis Date: Fri, 2 Aug 2024 23:55:20 +0300 Subject: [PATCH 16/38] SOLR-16824: Replace deprecated single-dash arguments (#2577) Migrate dash patterns in documentation, various tests, and various user messages generated by tools to reflect the move to --kebab-case and double dashes. --------- Co-authored-by: Eric Pugh --- solr/CHANGES.txt | 2 +- solr/bin/solr.cmd | 8 ++--- .../java/org/apache/solr/cli/CreateTool.java | 2 +- .../java/org/apache/solr/cli/PackageTool.java | 7 +++-- .../solr/packagemanager/PackageManager.java | 4 +-- .../org/apache/solr/cli/AuthToolTest.java | 2 +- .../org/apache/solr/cli/CreateToolTest.java | 2 +- .../org/apache/solr/cli/DeleteToolTest.java | 2 +- .../org/apache/solr/cli/PostToolTest.java | 4 +-- .../apache/solr/cli/SolrCLIZkToolsTest.java | 5 ++-- .../org/apache/solr/cli/TestExportTool.java | 2 +- .../apache/solr/cli/ZkSubcommandsTest.java | 4 +-- .../util/HdfsRecoverLeaseFileSystemUtils.java | 2 +- .../test/test_adminconsole_urls.bats | 2 +- solr/packaging/test/test_basic_auth.bats | 8 ++--- .../test/test_create_collection.bats | 2 +- solr/packaging/test/test_extraction.bats | 6 ++-- solr/packaging/test/test_packages.bats | 6 ++-- solr/packaging/test/test_post.bats | 4 +-- solr/packaging/test/test_prometheus.bats | 2 +- solr/packaging/test/test_ssl.bats | 2 +- solr/packaging/test/test_zk.bats | 2 +- .../pages/package-manager.adoc | 10 +++---- .../deployment-guide/pages/enabling-ssl.adoc | 6 ++-- .../pages/solr-control-script-reference.adoc | 30 +++++++++---------- .../pages/taking-solr-to-production.adoc | 2 +- .../pages/zookeeper-utilities.adoc | 2 +- .../getting-started/pages/tutorial-films.adoc | 2 +- .../pages/indexing-with-tika.adoc | 12 ++++---- .../indexing-guide/pages/post-tool.adoc | 14 ++++----- .../query-guide/pages/spatial-search.adoc | 4 +-- .../query-guide/pages/tagger-handler.adoc | 4 +-- .../solr/common/cloud/ZkMaintenanceUtils.java | 2 +- 33 files changed, 84 insertions(+), 84 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 99533e5f97c..27874851fe3 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -181,7 +181,7 @@ Improvements * SOLR-16824: Adopt Linux standard pattern of -- for long option commands, and make all commands "kebab" formatting. I.e -zkHost is now -zk-host. The old parameters such as -zkHost continue to be supported in the 9.x line of Solr. -u is now used to specify user credentials everywhere, this only impacts the bin/solr assert - commands "same user" check which has -u as the short form of --same-user. (Eric Pugh, janhoy, Jason Gerlowski) + commands "same user" check which has -u as the short form of --same-user. (Eric Pugh, janhoy, Jason Gerlowski, Christos Malliaridis) * SOLR-17346: Synchronise stopwords from snowball with those in Lucene (Alastair Porter via Houston Putman) diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd index 0eb25adce14..58fb4e6599e 100755 --- a/solr/bin/solr.cmd +++ b/solr/bin/solr.cmd @@ -246,6 +246,7 @@ set FIRST_ARG=%1 IF [%1]==[] goto usage +REM -help is a special case to faciliate folks learning about how to use Solr. IF "%1"=="-help" goto run_solrcli IF "%1"=="-usage" goto run_solrcli IF "%1"=="-h" goto run_solrcli @@ -293,7 +294,6 @@ IF "%FIRST_ARG%"=="-help" goto run_solrcli IF "%FIRST_ARG%"=="-usage" goto run_solrcli IF "%FIRST_ARG%"=="-h" goto run_solrcli IF "%FIRST_ARG%"=="--help" goto run_solrcli -IF "%FIRST_ARG%"=="-help" goto run_solrcli IF "%FIRST_ARG%"=="/?" goto run_solrcli IF "%SCRIPT_CMD%"=="start" goto start_usage IF "%SCRIPT_CMD%"=="restart" goto start_usage @@ -316,7 +316,7 @@ goto done @echo -f Start Solr in foreground; default starts Solr in the background @echo and sends stdout / stderr to solr-PORT-console.log @echo. -@echo -c or -cloud Start Solr in SolrCloud mode; if -z not supplied and ZK_HOST not defined in +@echo -c or --cloud Start Solr in SolrCloud mode; if -z not supplied and ZK_HOST not defined in @echo solr.in.cmd, an embedded ZooKeeper instance is started on Solr port+1000, @echo such as 9983 if Solr is bound to 8983 @echo. @@ -1493,10 +1493,10 @@ IF "!ZK_OP!"=="" ( set CONNECTION_PARAMS="" IF "!ZK_OP!"=="" ( - set CONNECTION_PARAMS="-solrUrl !ZK_SOLR_URL!" + set CONNECTION_PARAMS="--solr-url !ZK_SOLR_URL!" ) ELSE ( - set CONNECTION_PARAMS="-zkHost ZK_HOST!" + set CONNECTION_PARAMS="--zk-host ZK_HOST!" ) IF "!ZK_OP!"=="upconfig" ( diff --git a/solr/core/src/java/org/apache/solr/cli/CreateTool.java b/solr/core/src/java/org/apache/solr/cli/CreateTool.java index f828ac77423..61031ae5cc9 100644 --- a/solr/core/src/java/org/apache/solr/cli/CreateTool.java +++ b/solr/core/src/java/org/apache/solr/cli/CreateTool.java @@ -411,7 +411,7 @@ private void printDefaultConfigsetWarningIfNecessary(CommandLine cli) { final String configCommand = String.format( Locale.ROOT, - "bin/solr config -c %s -solrUrl %s -action set-user-property -property update.autoCreateFields -value false", + "bin/solr config -c %s -s %s --action set-user-property --property update.autoCreateFields --value false", collectionName, solrUrl); echo( diff --git a/solr/core/src/java/org/apache/solr/cli/PackageTool.java b/solr/core/src/java/org/apache/solr/cli/PackageTool.java index a8fdd5afd18..c3f4a3a9e96 100644 --- a/solr/core/src/java/org/apache/solr/cli/PackageTool.java +++ b/solr/core/src/java/org/apache/solr/cli/PackageTool.java @@ -205,8 +205,9 @@ public void runImpl(CommandLine cli) throws Exception { : new String[] {}; packageManager.undeploy(packageName, collections, cli.hasOption("cluster")); } else { + printRed( - "Either specify -cluster to undeploy cluster level plugins or -collections to undeploy collection level plugins"); + "Either specify --cluster to undeploy cluster level plugins or -collections to undeploy collection level plugins"); } break; } @@ -259,7 +260,7 @@ public String getHeader() { format(sb, ""); formatGreen( sb, - "bin/solr package deploy [:] [-y] [--update] -collections [-p = -p = ...] "); + "bin/solr package deploy [:] [-y] [--update] --collections [-p = -p = ...] "); format( sb, "Bootstraps a previously installed package into the specified collections. It the package accepts parameters for its setup commands, they can be specified (as per package documentation)."); @@ -277,7 +278,7 @@ public String getHeader() { format(sb, "Print a list of collections on which a given package has been deployed."); format(sb, ""); formatGreen( - sb, "bin/solr package undeploy -collections "); + sb, "bin/solr package undeploy --collections "); format(sb, "Undeploy a package from specified collection(s)"); format(sb, ""); formatGreen(sb, "bin/solr package uninstall :"); diff --git a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java index 1b671c0e172..15b6def6ecb 100644 --- a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java +++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java @@ -121,7 +121,7 @@ public void uninstall(String packageName, String version) + packageName + " is currently deployed on collection: " + collection - + ". Undeploy the package with undeploy -collections [,,...] before attempting to uninstall the package."); + + ". Undeploy the package with undeploy --collections [,,...] before attempting to uninstall the package."); System.exit(1); } } @@ -138,7 +138,7 @@ public void uninstall(String packageName, String version) + packageName + "is currently deployed as a cluster-level plugin (" + clusterPackageInstance.getCustomData() - + "). Undeploy the package with undeploy -collections [,,...] before uninstalling the package."); + + "). Undeploy the package with undeploy --collections [,,...] before uninstalling the package."); System.exit(1); } } diff --git a/solr/core/src/test/org/apache/solr/cli/AuthToolTest.java b/solr/core/src/test/org/apache/solr/cli/AuthToolTest.java index 60f5271c12f..8c32db28026 100644 --- a/solr/core/src/test/org/apache/solr/cli/AuthToolTest.java +++ b/solr/core/src/test/org/apache/solr/cli/AuthToolTest.java @@ -70,7 +70,7 @@ public void testEnableAuth() throws Exception { dir.toAbsolutePath().toString(), "--solr-include-file", solrIncludeFile.toAbsolutePath().toString(), - "-credentials", + "--credentials", "solr:solr", "--block-unknown", "true" diff --git a/solr/core/src/test/org/apache/solr/cli/CreateToolTest.java b/solr/core/src/test/org/apache/solr/cli/CreateToolTest.java index 333b541f7e6..838c3857ce9 100644 --- a/solr/core/src/test/org/apache/solr/cli/CreateToolTest.java +++ b/solr/core/src/test/org/apache/solr/cli/CreateToolTest.java @@ -49,7 +49,7 @@ public void testCreateCollectionWithBasicAuth() throws Exception { "cloud-minimal", "-z", cluster.getZkClient().getZkServerAddress(), - "-credentials", + "--credentials", SecurityJson.USER_PASS, "-verbose" }; diff --git a/solr/core/src/test/org/apache/solr/cli/DeleteToolTest.java b/solr/core/src/test/org/apache/solr/cli/DeleteToolTest.java index 535ffc75573..86459b1f71a 100644 --- a/solr/core/src/test/org/apache/solr/cli/DeleteToolTest.java +++ b/solr/core/src/test/org/apache/solr/cli/DeleteToolTest.java @@ -64,7 +64,7 @@ public void testDeleteCollectionWithBasicAuth() throws Exception { "false", "-z", cluster.getZkClient().getZkServerAddress(), - "-credentials", + "--credentials", SecurityJson.USER_PASS, "-verbose" }; diff --git a/solr/core/src/test/org/apache/solr/cli/PostToolTest.java b/solr/core/src/test/org/apache/solr/cli/PostToolTest.java index 1f507b281b5..6f7d167e106 100644 --- a/solr/core/src/test/org/apache/solr/cli/PostToolTest.java +++ b/solr/core/src/test/org/apache/solr/cli/PostToolTest.java @@ -127,7 +127,7 @@ public void testRunWithCollectionParam() throws Exception { fw.flush(); String[] args = { - "post", "-c", collection, "-credentials", SecurityJson.USER_PASS, jsonDoc.getAbsolutePath() + "post", "-c", collection, "--credentials", SecurityJson.USER_PASS, jsonDoc.getAbsolutePath() }; assertEquals(0, runTool(args)); @@ -167,7 +167,7 @@ public void testRunCsvWithCustomSeparatorParam() throws Exception { "post", "-c", collection, - "-credentials", + "--credentials", SecurityJson.USER_PASS, "--params", "\"separator=%09&header=false&fieldnames=id,title_s\"", diff --git a/solr/core/src/test/org/apache/solr/cli/SolrCLIZkToolsTest.java b/solr/core/src/test/org/apache/solr/cli/SolrCLIZkToolsTest.java index 3da316a1702..2470d01dbf8 100644 --- a/solr/core/src/test/org/apache/solr/cli/SolrCLIZkToolsTest.java +++ b/solr/core/src/test/org/apache/solr/cli/SolrCLIZkToolsTest.java @@ -768,7 +768,8 @@ public void testRm() throws Exception { int res = tool.runTool(SolrCLI.processCommandLineArgs(tool, args)); assertTrue( - "Should have failed to remove node with children unless -recurse is set to true", res != 0); + "Should have failed to remove node with children unless --recurse is set to true", + res != 0); // Are we sure all the znodes are still there? verifyZkLocalPathsMatch(srcPathCheck, "/configs/rm1"); @@ -783,7 +784,7 @@ public void testRm() throws Exception { res = tool.runTool(SolrCLI.processCommandLineArgs(tool, args)); assertTrue( - "Should have failed to remove node with children if -recurse is set to false", res != 0); + "Should have failed to remove node with children if --recurse is set to false", res != 0); args = new String[] { diff --git a/solr/core/src/test/org/apache/solr/cli/TestExportTool.java b/solr/core/src/test/org/apache/solr/cli/TestExportTool.java index 1223925058b..357fd26d690 100644 --- a/solr/core/src/test/org/apache/solr/cli/TestExportTool.java +++ b/solr/core/src/test/org/apache/solr/cli/TestExportTool.java @@ -245,7 +245,7 @@ public void testWithBasicAuth() throws Exception { "export", "-url", cluster.getJettySolrRunner(0).getBaseUrl() + "/" + COLLECTION_NAME, - "-credentials", + "--credentials", SecurityJson.USER_PASS, "-out", outFile.getAbsolutePath(), diff --git a/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java b/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java index 8185c5b1457..250cee1a955 100644 --- a/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java +++ b/solr/core/src/test/org/apache/solr/cli/ZkSubcommandsTest.java @@ -635,14 +635,14 @@ public void testSetClusterProperty() throws Exception { // add property urlScheme=http String[] args = new String[] { - "cluster", "-property", "urlScheme", "-value", "http", "-z", zkServer.getZkAddress() + "cluster", "--property", "urlScheme", "--value", "http", "-z", zkServer.getZkAddress() }; ClusterTool tool = new ClusterTool(); assertEquals(0, runTool(args, tool)); assertEquals("http", properties.getClusterProperty("urlScheme", "none")); - args = new String[] {"cluster", "-property", "urlScheme", "-z", zkServer.getZkAddress()}; + args = new String[] {"cluster", "--property", "urlScheme", "-z", zkServer.getZkAddress()}; assertEquals(0, runTool(args, tool)); assertNull(properties.getClusterProperty("urlScheme", (String) null)); } diff --git a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/util/HdfsRecoverLeaseFileSystemUtils.java b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/util/HdfsRecoverLeaseFileSystemUtils.java index b0bd7960bff..2d5cd6a06ac 100644 --- a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/util/HdfsRecoverLeaseFileSystemUtils.java +++ b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/util/HdfsRecoverLeaseFileSystemUtils.java @@ -54,7 +54,7 @@ public static void recoverFileLease( /* * Run the dfs recover lease. recoverLease is asynchronous. It returns: - * -false when it starts the lease recovery (i.e. lease recovery not *yet* done) + * - false when it starts the lease recovery (i.e. lease recovery not *yet* done) * - true when the lease recovery has succeeded or the file is closed. * But, we have to be careful. Each time we call recoverLease, it starts the recover lease * process over from the beginning. We could put ourselves in a situation where we are diff --git a/solr/packaging/test/test_adminconsole_urls.bats b/solr/packaging/test/test_adminconsole_urls.bats index 68fb79df3ed..2ab86af39cd 100644 --- a/solr/packaging/test/test_adminconsole_urls.bats +++ b/solr/packaging/test/test_adminconsole_urls.bats @@ -25,7 +25,7 @@ teardown() { # save a snapshot of SOLR_HOME for failed tests save_home_on_failure - solr stop -all >/dev/null 2>&1 + solr stop --all >/dev/null 2>&1 } @test "assert able to launch solr admin console" { diff --git a/solr/packaging/test/test_basic_auth.bats b/solr/packaging/test/test_basic_auth.bats index 8cd997a08ad..9cdebb7af18 100644 --- a/solr/packaging/test/test_basic_auth.bats +++ b/solr/packaging/test/test_basic_auth.bats @@ -49,15 +49,15 @@ teardown() { assert_output --partial "Created collection 'COLL_NAME'" # Test config - run solr config -u name:password -c COLL_NAME --action set-property -property updateHandler.autoCommit.maxDocs --value 100 --solr-url http://localhost:${SOLR_PORT}/solr + run solr config -u name:password -c COLL_NAME --action set-property --property updateHandler.autoCommit.maxDocs --value 100 --solr-url http://localhost:${SOLR_PORT}/solr assert_output --partial "Successfully set-property updateHandler.autoCommit.maxDocs to 100" # Test api - run solr api -u name:password --solr-url "http://localhost:${SOLR_PORT}/solr/COLL_NAME/select?q=*:*" -verbose + run solr api -u name:password --solr-url "http://localhost:${SOLR_PORT}/solr/COLL_NAME/select?q=*:*" --verbose assert_output --partial '"numFound":0' # Test delete - run solr delete --credentials name:password -c COLL_NAME -z localhost:${ZK_PORT} -verbose + run solr delete --credentials name:password -c COLL_NAME -z localhost:${ZK_PORT} --verbose assert_output --partial "Deleted collection 'COLL_NAME'" refute collection_exists "COLL_NAME" @@ -69,7 +69,7 @@ run solr create -c COLL_NAME assert_output --partial "Created collection 'COLL_NAME'" # Test post - run solr post -u name:password -type application/xml --solr-update-url http://localhost:${SOLR_PORT}/solr/monitors/update ${SOLR_TIP}/example/exampledocs/monitor.xml + run solr post -u name:password -t application/xml --solr-update-url http://localhost:${SOLR_PORT}/solr/monitors/update ${SOLR_TIP}/example/exampledocs/monitor.xml assert_output --partial '1 files indexed.' # Test postlogs diff --git a/solr/packaging/test/test_create_collection.bats b/solr/packaging/test/test_create_collection.bats index 05ec63d7f74..3ffa6bd0ff3 100644 --- a/solr/packaging/test/test_create_collection.bats +++ b/solr/packaging/test/test_create_collection.bats @@ -63,7 +63,7 @@ teardown() { } @test "reject d option with invalid config dir" { - run ! solr create -c COLL_NAME -d /asdf -solrUrl http://localhost:${SOLR_PORT} + run ! solr create -c COLL_NAME -d /asdf --solr-url http://localhost:${SOLR_PORT} assert_output --partial "Specified configuration directory /asdf not found!" } diff --git a/solr/packaging/test/test_extraction.bats b/solr/packaging/test/test_extraction.bats index 7557dc93b94..346f1df431a 100644 --- a/solr/packaging/test/test_extraction.bats +++ b/solr/packaging/test/test_extraction.bats @@ -73,7 +73,7 @@ teardown() { }' "http://localhost:${SOLR_PORT}/solr/content_extraction/config" # We filter to pdf to invoke the Extract handler. - run solr post -filetypes pdf -url http://localhost:${SOLR_PORT}/solr/content_extraction/update ${SOLR_TIP}/example/exampledocs + run solr post --filetypes pdf -url http://localhost:${SOLR_PORT}/solr/content_extraction/update ${SOLR_TIP}/example/exampledocs assert_output --partial '1 files indexed.' refute_output --partial 'ERROR' @@ -99,8 +99,8 @@ teardown() { } }' "http://localhost:${SOLR_PORT}/solr/website_extraction/config" - # Change to -recursive 1 to crawl multiple pages, but may be too slow. - run solr post -mode web -url http://localhost:${SOLR_PORT}/solr/website_extraction/update -recursive 0 -delay 1 https://solr.apache.org/ + # Change to --recursive 1 to crawl multiple pages, but may be too slow. + run solr post --mode web --solr-update-url http://localhost:${SOLR_PORT}/solr/website_extraction/update --recursive 0 --delay 1 https://solr.apache.org/ assert_output --partial 'POSTed web resource https://solr.apache.org (depth: 0)' refute_output --partial 'ERROR' diff --git a/solr/packaging/test/test_packages.bats b/solr/packaging/test/test_packages.bats index 185b3381b83..7de497a8d8c 100644 --- a/solr/packaging/test/test_packages.bats +++ b/solr/packaging/test/test_packages.bats @@ -44,7 +44,7 @@ teardown() { solr create -c foo-1.2 # Deploy package - the package doesn't need to exist before the collection validation kicks in - run solr package deploy PACKAGE_NAME -collections foo-1.2 + run solr package deploy PACKAGE_NAME --collections foo-1.2 # assert_output --partial "Deployment successful" refute_output --partial "Invalid collection" @@ -52,7 +52,7 @@ teardown() { assert_output --partial "Package instance doesn't exist: PACKAGE_NAME:null" # Undeploy package - run solr package undeploy PACKAGE_NAME -collections foo-1.2 + run solr package undeploy PACKAGE_NAME --collections foo-1.2 refute_output --partial "Invalid collection" assert_output --partial "Package PACKAGE_NAME not deployed on collection foo-1.2" } @@ -72,7 +72,7 @@ teardown() { # run solr package install solr-splainer # assert_output --partial "solr-splainer installed." -# run solr package deploy solr-splainer -y -cluster +# run solr package deploy solr-splainer -y --cluster # assert_output --partial "Deployment successful" # run -0 curl --fail http://localhost:${SOLR_PORT}/v2/splainer/index.html diff --git a/solr/packaging/test/test_post.bats b/solr/packaging/test/test_post.bats index bb9b2e6cb1a..38652a8c5da 100644 --- a/solr/packaging/test/test_post.bats +++ b/solr/packaging/test/test_post.bats @@ -162,14 +162,14 @@ teardown() { run solr create -c test_args -d _default assert_output --partial "Created collection 'test_args'" - run solr post --solr-update-url http://localhost:${SOLR_PORT}/solr/test_args/update --mode args -type application/xml --out "*:*" + run solr post --solr-update-url http://localhost:${SOLR_PORT}/solr/test_args/update --mode args --type application/xml --out "*:*" assert_output --partial '0' # confirm default type run solr post --solr-update-url http://localhost:${SOLR_PORT}/solr/test_args/update --mode args --out "{'delete': {'query': '*:*'}}" assert_output --partial '"status":0' - # confirm we don't get back output without -out + # confirm we don't get back output without --out run solr post --solr-update-url http://localhost:${SOLR_PORT}/solr/test_args/update --mode args "{'delete': {'query': '*:*'}}" refute_output --partial '"status":0' diff --git a/solr/packaging/test/test_prometheus.bats b/solr/packaging/test/test_prometheus.bats index 72dbb3b4690..9af9a515922 100644 --- a/solr/packaging/test/test_prometheus.bats +++ b/solr/packaging/test/test_prometheus.bats @@ -25,7 +25,7 @@ teardown() { # save a snapshot of SOLR_HOME for failed tests save_home_on_failure - solr stop -all >/dev/null 2>&1 + solr stop --all >/dev/null 2>&1 shutdown_exporter } diff --git a/solr/packaging/test/test_ssl.bats b/solr/packaging/test/test_ssl.bats index 08b3fc1d786..d114c15e25d 100644 --- a/solr/packaging/test/test_ssl.bats +++ b/solr/packaging/test/test_ssl.bats @@ -604,7 +604,7 @@ teardown() { run solr api --solr-url "https://localhost:${SOLR2_PORT}/solr/test-single-shard/select?q=query4" assert_output --partial '"numFound":0' - run solr post -url https://localhost:${SOLR_PORT}/solr/test/update ${SOLR_TIP}/example/exampledocs/books.csv + run solr post --solr-update-url https://localhost:${SOLR_PORT}/solr/test/update ${SOLR_TIP}/example/exampledocs/books.csv run solr api --solr-url "https://localhost:${SOLR_PORT}/solr/test/select?q=*:*" assert_output --partial '"numFound":10' diff --git a/solr/packaging/test/test_zk.bats b/solr/packaging/test/test_zk.bats index 1c2c2a2bad7..817a33e1054 100644 --- a/solr/packaging/test/test_zk.bats +++ b/solr/packaging/test/test_zk.bats @@ -122,7 +122,7 @@ teardown() { @test "bin/solr zk cp gets 'solrhome' from '--solr-home' command line option" { touch afile.txt - run solr zk cp afile.txt zk:/afile.txt -z localhost:${ZK_PORT} -verbose --solr-home ${SOLR_TIP}/server/solr + run solr zk cp afile.txt zk:/afile.txt -z localhost:${ZK_PORT} --verbose --solr-home ${SOLR_TIP}/server/solr assert_output --partial "Using SolrHome: ${SOLR_TIP}/server/solr" refute_output --partial 'Failed to load solr.xml from ZK or SolrHome' diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/package-manager.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/package-manager.adoc index f7e37aa77c4..d2bbc8a279a 100644 --- a/solr/solr-ref-guide/modules/configuration-guide/pages/package-manager.adoc +++ b/solr/solr-ref-guide/modules/configuration-guide/pages/package-manager.adoc @@ -109,7 +109,7 @@ or [source,bash] ---- -$ bin/solr package deploy :[version] -cluster +$ bin/solr package deploy :[version] --cluster ---- If the package accepts parameters for its setup commands, they can be specified (as per package documentation): @@ -170,12 +170,12 @@ Assuming the old version is `1.0.0` of the package `mypackage`, and the new vers [source,bash] ---- -$ bin/solr package deploy mypackage:2.0.0 --update -collections mycollection +$ bin/solr package deploy mypackage:2.0.0 --update --collections mycollection ---- or [source,bash] ---- -$ bin/solr package deploy mypackage:2.0.0 --update -cluster +$ bin/solr package deploy mypackage:2.0.0 --update --cluster ---- You can run the `list-deployed` command to verify that this collection is using the newly added version. @@ -186,7 +186,7 @@ If a package supports undeploying the plugins it contains (check package author' [source,bash] ---- -$ bin/solr package undeploy -collections [,,...] +$ bin/solr package undeploy --collections [,,...] ---- === Uninstall a Package @@ -202,7 +202,7 @@ or [source,bash] ---- -$ bin/solr package deploy : -cluster +$ bin/solr package deploy : --cluster ---- Both package name and version are required. diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc index 64eac73e73d..db5880049e0 100644 --- a/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc +++ b/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc @@ -234,7 +234,7 @@ The examples below use the `zkcli` tool that comes with Solr to do this. ==== [source,terminal] ---- -$ bin/solr cluster -property urlSchema -value https -z server1:2181,server2:2181,server3:2181 +$ bin/solr cluster --property urlSchema --value https -z server1:2181,server2:2181,server3:2181 ---- ==== @@ -243,13 +243,13 @@ Windows:: ==== [source,powershell] ---- -C:\> bin/solr.cmd -property urlSchema -value https -z server1:2181,server2:2181,server3:2181 +C:\> bin/solr.cmd --property urlSchema --value https -z server1:2181,server2:2181,server3:2181 ---- ==== ====== Be sure to use the correct `zkhost` value for your system. -If you have set up your ZooKeeper ensemble to use a xref:taking-solr-to-production.adoc#zookeeper-chroot[chroot for Solr], make sure to include it in the `zkhost` string, e.g., `-zkhost server1:2181,server2:2181,server3:2181/solr`. +If you have set up your ZooKeeper ensemble to use a xref:taking-solr-to-production.adoc#zookeeper-chroot[chroot for Solr], make sure to include it in the `zkHost` string, e.g., `--zk-host server1:2181,server2:2181,server3:2181/solr`. === Update Cluster Properties for Existing Collections diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/solr-control-script-reference.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/solr-control-script-reference.adoc index 9a0d0f87b42..3a517224747 100644 --- a/solr/solr-ref-guide/modules/deployment-guide/pages/solr-control-script-reference.adoc +++ b/solr/solr-ref-guide/modules/deployment-guide/pages/solr-control-script-reference.adoc @@ -248,7 +248,7 @@ This can be useful in a production setting where you want to limit logging to wa + *Example*: `bin/solr start -f -q` -`-V` or `-verbose`:: +`-V` or `--verbose`:: + [%autowidth,frame=none] |=== @@ -273,7 +273,7 @@ Set the `ZK_CREATE_CHROOT` environment variable to true if your ZK host has a ch + *Example*: `bin/solr start -c -z server1:2181,server2:2181` -`-force`:: +`--force`:: + [%autowidth,frame=none] |=== @@ -281,9 +281,9 @@ Set the `ZK_CREATE_CHROOT` environment variable to true if your ZK host has a ch |=== + If attempting to start Solr as the root user, the script will exit with a warning that running Solr as "root" can cause problems. -It is possible to override this warning with the `-force` parameter. +It is possible to override this warning with the `--force` parameter. + -*Example*: `sudo bin/solr start -force` +*Example*: `sudo bin/solr start --force` To emphasize how the default settings work take a moment to understand that the following commands are equivalent: @@ -400,7 +400,7 @@ The command will wait up to 180 seconds for Solr to stop gracefully and then wil |=== + Stop Solr running on the given port. -If you are running more than one instance, or are running in SolrCloud mode, you either need to specify the ports in separate requests or use the `-all` option. +If you are running more than one instance, or are running in SolrCloud mode, you either need to specify the ports in separate requests or use the `--all` option. + *Example*: `bin/solr stop -p 8983` @@ -426,7 +426,7 @@ Stop key used to protect from stopping Solr inadvertently; default is "solrrocks + *Example*: `bin/solr stop -k solrrocks` -`-V` or `-verbose`:: +`-V` or `--verbose`:: + [%autowidth,frame=none] |=== @@ -720,7 +720,7 @@ The default is `1` (no replication). + *Example*: `bin/solr create -rf 2` -`-force`:: +`--force`:: + [%autowidth,frame=none] |=== @@ -728,11 +728,11 @@ The default is `1` (no replication). |=== + If attempting to run create as "root" user, the script will exit with a warning that running Solr or actions against Solr as "root" can cause problems. -It is possible to override this warning with the -force parameter. +It is possible to override this warning with the --force parameter. + *Example*: `bin/solr create -c foo --force` -`-z ` or `-zk-host `:: +`-z ` or `--zk-host `:: + [%autowidth,frame=none] |=== @@ -935,10 +935,10 @@ The command takes the following parameters: The username and password in the format of `username:password` of the initial user. Applicable for basicAuth only. + -If you prefer not to pass the username and password as an argument to the script, you can choose the `-prompt` option. -Either `-credentials` or `-prompt` *must* be specified. +If you prefer not to pass the username and password as an argument to the script, you can choose the `--prompt` option. +Either `--credentials` or `--prompt` *must* be specified. -`-prompt `:: +`--prompt `:: + [%autowidth,frame=none] |=== @@ -947,7 +947,7 @@ Either `-credentials` or `-prompt` *must* be specified. + Prompts the user to provide the credentials. If prompt is preferred, pass `true` as a parameter to request the script prompt the user to enter a username and password. + -Either `-credentials` or `-prompt` *must* be specified. +Either `--credentials` or `--prompt` *must* be specified. `--block-unknown `:: + @@ -959,7 +959,7 @@ Either `-credentials` or `-prompt` *must* be specified. When `true`, this blocks out access to unauthenticated users from accessing Solr. When `false`, unauthenticated users will still be able to access Solr, but only for operations not explicitly requiring a user role in the Authorization plugin configuration. -`-solrIncludeFile `:: +`--solrIncludeFile `:: + [%autowidth,frame=none] |=== @@ -986,7 +986,7 @@ If not specified this script looks for an include file named solr.in.sh to set e + When `true`, only update the `bin/solr.in.sh` or `bin\solr.in.cmd`, and skip actual enabling/disabling authentication (i.e. don't update `security.json`). -`-z ` or `-zk-host `:: +`-z ` or `--zk-host `:: + [%autowidth,frame=none] |=== diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/taking-solr-to-production.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/taking-solr-to-production.adoc index a67a49a9e81..4adb5fd6b57 100644 --- a/solr/solr-ref-guide/modules/deployment-guide/pages/taking-solr-to-production.adoc +++ b/solr/solr-ref-guide/modules/deployment-guide/pages/taking-solr-to-production.adoc @@ -93,7 +93,7 @@ To see available options, simply do: [source,bash] ---- -sudo bash ./install_solr_service.sh -help +sudo bash ./install_solr_service.sh --help ---- Once the script completes, Solr will be installed as a service and running in the background on your server (on port 8983). diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-utilities.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-utilities.adoc index 2b08e1aea3a..864ccb1b1dd 100644 --- a/solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-utilities.adoc +++ b/solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-utilities.adoc @@ -30,7 +30,7 @@ Use the `help` option to get a list of available ZooKeeper specifc commands from == Solr CLI Examples -Below are some examples of using the `bin/solr` CLI which assume you have already started the SolrCloud example (`bin/solr start -e cloud -noprompt`) +Below are some examples of using the `bin/solr` CLI which assume you have already started the SolrCloud example (`bin/solr start -e cloud --no-prompt`) === Upload a Configuration Directory diff --git a/solr/solr-ref-guide/modules/getting-started/pages/tutorial-films.adoc b/solr/solr-ref-guide/modules/getting-started/pages/tutorial-films.adoc index 8191682735c..69622b5c7cc 100644 --- a/solr/solr-ref-guide/modules/getting-started/pages/tutorial-films.adoc +++ b/solr/solr-ref-guide/modules/getting-started/pages/tutorial-films.adoc @@ -243,7 +243,7 @@ $ bin/solr post -c films example/films/films.xml ====== [,console] ---- -$ bin/solr post -c films example/films/films.csv -params "f.genre.split=true&f.directed_by.split=true&f.genre.separator=|&f.directed_by.separator=|" +$ bin/solr post -c films example/films/films.csv --params "f.genre.split=true&f.directed_by.split=true&f.genre.separator=|&f.directed_by.separator=|" ---- ====== diff --git a/solr/solr-ref-guide/modules/indexing-guide/pages/indexing-with-tika.adoc b/solr/solr-ref-guide/modules/indexing-guide/pages/indexing-with-tika.adoc index 8800d85c018..14b26ed556d 100644 --- a/solr/solr-ref-guide/modules/indexing-guide/pages/indexing-with-tika.adoc +++ b/solr/solr-ref-guide/modules/indexing-guide/pages/indexing-with-tika.adoc @@ -128,7 +128,7 @@ You can also use `bin/solr post` to do the same thing: [,console] ---- -$ bin/solr post -c gettingstarted example/exampledocs/solr-word.pdf -params "literal.id=doc1" +$ bin/solr post -c gettingstarted example/exampledocs/solr-word.pdf --params "literal.id=doc1" ---- Now you can execute a query and find that document with a request like `\http://localhost:8983/solr/gettingstarted/select?q=pdf`. @@ -148,7 +148,7 @@ Here's an example: [,console] ---- -$ bin/solr post -c gettingstarted example/exampledocs/solr-word.pdf -params "literal.id=doc1&uprefix=ignored_&fmap.last_modified=last_modified_dt" +$ bin/solr post -c gettingstarted example/exampledocs/solr-word.pdf --params "literal.id=doc1&uprefix=ignored_&fmap.last_modified=last_modified_dt" ---- [NOTE] @@ -563,7 +563,7 @@ The command below captures `

` tags separately (`capture=div`), and then map [,console] ---- -$ bin/solr post -c gettingstarted example/exampledocs/sample.html -params "literal.id=doc2&captureAttr=true&defaultField=_text_&fmap.div=foo_t&capture=div" +$ bin/solr post -c gettingstarted example/exampledocs/sample.html --params "literal.id=doc2&captureAttr=true&defaultField=_text_&fmap.div=foo_t&capture=div" ---- === Using Literals to Define Custom Metadata @@ -572,7 +572,7 @@ To add in your own metadata, pass in the literal parameter along with the file: [,console] ---- -$ bin/solr post -c gettingstarted -params "literal.id=doc4&captureAttr=true&defaultField=text&capture=div&fmap.div=foo_t&literal.blah_s=Bah" example/exampledocs/sample.html +$ bin/solr post -c gettingstarted --params "literal.id=doc4&captureAttr=true&defaultField=text&capture=div&fmap.div=foo_t&literal.blah_s=Bah" example/exampledocs/sample.html ---- The parameter `literal.blah_s=Bah` will insert a field `blah_s` into every document. @@ -584,7 +584,7 @@ The example below passes in an XPath expression to restrict the XHTML returned b [,console] ---- -$ bin/solr post -c gettingstarted -params "literal.id=doc5&captureAttr=true&defaultField=text&capture=div&fmap.div=foo_t&xpath=/xhtml:html/xhtml:body/xhtml:div//node()" example/exampledocs/sample.html +$ bin/solr post -c gettingstarted --params "literal.id=doc5&captureAttr=true&defaultField=text&capture=div&fmap.div=foo_t&xpath=/xhtml:html/xhtml:body/xhtml:div//node()" example/exampledocs/sample.html ---- === Extracting Data without Indexing @@ -603,7 +603,7 @@ The output includes XML generated by Tika (and further escaped by Solr's XML) us [,console] ---- -$ bin/solr post -c gettingstarted -params "extractOnly=true&wt=ruby&indent=true" -out yes example/exampledocs/sample.html +$ bin/solr post -c gettingstarted --params "extractOnly=true&wt=ruby&indent=true" --out yes example/exampledocs/sample.html ---- === Using Solr Cell with a POST Request diff --git a/solr/solr-ref-guide/modules/indexing-guide/pages/post-tool.adoc b/solr/solr-ref-guide/modules/indexing-guide/pages/post-tool.adoc index b6cd39a86fa..7d904337aad 100644 --- a/solr/solr-ref-guide/modules/indexing-guide/pages/post-tool.adoc +++ b/solr/solr-ref-guide/modules/indexing-guide/pages/post-tool.adoc @@ -135,7 +135,7 @@ Send XML arguments to delete a document from `gettingstarted`. [,console] ---- -$ bin/solr post -url http://localhost:8983/solr/gettingstarted/update -mode args -type application/xml '42' +$ bin/solr post -url http://localhost:8983/solr/gettingstarted/update --mode args --type application/xml '42' ---- === Indexing CSV and JSON @@ -144,14 +144,14 @@ Index all CSV and JSON files into `gettingstarted` from current directory: [,console] ---- -$ bin/solr post -c gettingstarted -filetypes json,csv . +$ bin/solr post -c gettingstarted --filetypes json,csv . ---- Index a tab-separated file into `gettingstarted`: [,console] ---- -$ bin/solr post -url http://localhost:8984/solr/signals/update -params "separator=%09" -type text/csv data.tsv +$ bin/solr post -url http://localhost:8984/solr/signals/update --params "separator=%09" --type text/csv data.tsv ---- The content type (`-type`) parameter is required to treat the file as the proper type, otherwise it will be ignored and a WARNING logged as it does not know what type of content a .tsv file is. @@ -177,7 +177,7 @@ Automatically detect content types in a folder, but limit it to PPT and HTML fil [,console] ---- -$ bin/solr post -url http://localhost:8983/solr/gettingstarted/update -filetypes ppt,html afolder/ +$ bin/solr post -url http://localhost:8983/solr/gettingstarted/update --filetypes ppt,html afolder/ ---- === Indexing to a Password Protected Solr (Basic Auth) @@ -197,7 +197,7 @@ See xref:indexing-with-tika.adoc#trying-out-solr-cell[Trying Out Solr Cell] to l [,console] ---- -$ bin/solr post -mode web -c gettingstarted -recursive 1 -delay 1 https://solr.apache.org/ +$ bin/solr post --mode web -c gettingstarted --recursive 1 --delay 1 https://solr.apache.org/ ---- === Standard Input as Source for Indexing @@ -207,7 +207,7 @@ Notice the `-out` providing raw responses from Solr. [,console] ---- -$ echo '{commit: {}}' | bin/solr post -mode stdin -url http://localhost:8983/my_collection/update -out +$ echo '{commit: {}}' | bin/solr post --mode stdin -url http://localhost:8983/my_collection/update --out ---- === Raw Data as Source for Indexing @@ -216,5 +216,5 @@ Provide the raw document as a string for indexing. [,console] ---- -$ bin/solr post -url http://localhost:8983/signals/update -mode args -type text/csv -out $'id,value\n1,0.47' +$ bin/solr post -url http://localhost:8983/signals/update -mode args --type text/csv -out $'id,value\n1,0.47' ---- diff --git a/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc b/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc index cb714e2f8ac..68fba64f500 100644 --- a/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc +++ b/solr/solr-ref-guide/modules/query-guide/pages/spatial-search.adoc @@ -71,9 +71,7 @@ However, it's much bulkier than the raw coordinates for such simple data. Using the `bin/solr post` tool: [,console] ----- -$ bin/solr post -type "application/json" -url "http://localhost:8983/solr/mycollection/update?format=geojson" /path/to/geojson.file ----- +$ bin/solr post -t "application/json" -url "http://localhost:8983/solr/mycollection/update?format=geojson" /path/to/geojson.file The key parameter to pass in with your request is: diff --git a/solr/solr-ref-guide/modules/query-guide/pages/tagger-handler.adoc b/solr/solr-ref-guide/modules/query-guide/pages/tagger-handler.adoc index 7ec9849d24c..2cb2e0589fe 100644 --- a/solr/solr-ref-guide/modules/query-guide/pages/tagger-handler.adoc +++ b/solr/solr-ref-guide/modules/query-guide/pages/tagger-handler.adoc @@ -277,8 +277,8 @@ population. Using bin/solr post: [,console] ---- -$ bin/solr post -c geonames -type text/csv \ - -params 'optimize=true&maxSegments=1&separator=%09&encapsulator=%00&fieldnames=id,name,,alternative_names,latitude,longitude,,,countrycode,,,,,,population,elevation,,timezone,lastupdate' \ +$ bin/solr post -c geonames -t text/csv \ + --params 'optimize=true&maxSegments=1&separator=%09&encapsulator=%00&fieldnames=id,name,,alternative_names,latitude,longitude,,,countrycode,,,,,,population,elevation,,timezone,lastupdate' \ /tmp/cities1000.txt ---- diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java index 7acb40eb3b5..a3fe06f4a11 100644 --- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java +++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java @@ -137,7 +137,7 @@ public static void zkTransfer( throw new SolrServerException("One or both of source or destination must specify ZK nodes."); } - // Make sure -recurse is specified if the source has children. + // Make sure --recurse is specified if the source has children. if (recurse == false) { if (srcIsZk) { if (zkClient.getChildren(src, null, true).size() != 0) { From 759b94fcae81ef00ed245142c0ef513814930ab4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rafa=C5=82=20Harabie=C5=84?= Date: Sat, 3 Aug 2024 06:53:21 +0200 Subject: [PATCH 17/38] SOLR-17276: Prometheus Exporter: scrape metrics at fixed rate (#2442) --- solr/CHANGES.txt | 2 ++ .../solr/prometheus/collector/SchedulerMetricsCollector.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 27874851fe3..f7c8fa4d9ae 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -151,6 +151,8 @@ New Features * SOLR-10255: Add support for docValues to solr.BinaryField. (Alexey Serba via Mikhail Khludnev, David Smiley) +* SOLR-17276: Prometheus Exporter: now scrapes metrics at a fixed rate instead of delay. (Rafał Harabień) + Improvements --------------------- * SOLR-10808, SOLR-12963: The Solr schema version has been increased to 1.7. diff --git a/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/collector/SchedulerMetricsCollector.java b/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/collector/SchedulerMetricsCollector.java index 8a1a3f60638..26fe8110c7b 100644 --- a/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/collector/SchedulerMetricsCollector.java +++ b/solr/prometheus-exporter/src/java/org/apache/solr/prometheus/collector/SchedulerMetricsCollector.java @@ -74,7 +74,7 @@ public SchedulerMetricsCollector( } public void start() { - scheduler.scheduleWithFixedDelay(this::collectMetrics, 0, duration, timeUnit); + scheduler.scheduleAtFixedRate(this::collectMetrics, 0, duration, timeUnit); } @SuppressWarnings("try") From 14375b621fe3c33c20f7a1a0e1ad6f4caffd8f21 Mon Sep 17 00:00:00 2001 From: Matthew Biscocho <54160956+mlbiscoc@users.noreply.github.com> Date: Sat, 3 Aug 2024 15:04:17 -0400 Subject: [PATCH 18/38] SOLR-17368: Fix TestPrometheusResponseWriter by clearing registries (#2613) SharedMetricRegistries.clear() Parse numeric value based on if tags exist Co-authored-by: Matthew Biscocho Co-authored-by: David Smiley --- .../response/TestPrometheusResponseWriter.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java index 8cf651e3928..0e25201cce0 100644 --- a/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java +++ b/solr/core/src/test/org/apache/solr/response/TestPrometheusResponseWriter.java @@ -20,6 +20,7 @@ import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; import com.codahale.metrics.SettableGauge; +import com.codahale.metrics.SharedMetricRegistries; import java.lang.invoke.MethodHandles; import java.util.Arrays; import java.util.List; @@ -34,6 +35,7 @@ import org.apache.solr.metrics.SolrMetricManager; import org.apache.solr.util.ExternalPaths; import org.apache.solr.util.SolrJettyTestRule; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; @@ -47,6 +49,8 @@ public class TestPrometheusResponseWriter extends SolrTestCaseJ4 { @BeforeClass public static void beforeClass() throws Exception { + SharedMetricRegistries.clear(); + solrClientTestRule.startSolr(LuceneTestCase.createTempDir()); solrClientTestRule.newCollection().withConfigSet(ExternalPaths.DEFAULT_CONFIGSET).create(); var cc = solrClientTestRule.getCoreContainer(); @@ -62,6 +66,11 @@ public static void beforeClass() throws Exception { registerGauge(manager, "solr.jvm", "gc.dummyMetrics.count"); } + @AfterClass + public static void clearMetricsRegistries() { + SharedMetricRegistries.clear(); + } + @Test public void testPrometheusStructureOutput() throws Exception { ModifiableSolrParams params = new ModifiableSolrParams(); @@ -78,7 +87,12 @@ public void testPrometheusStructureOutput() throws Exception { output.lines().filter(line -> !line.startsWith("#")).collect(Collectors.toList()); filteredResponse.forEach( (actualMetric) -> { - String actualValue = actualMetric.substring(actualMetric.lastIndexOf("} ") + 1); + String actualValue; + if (actualMetric.contains("}")) { + actualValue = actualMetric.substring(actualMetric.lastIndexOf("} ") + 1); + } else { + actualValue = actualMetric.split(" ")[1]; + } assertTrue( "All metrics should start with 'solr_metrics_'", actualMetric.startsWith("solr_metrics_")); From 988e9e3c2666784638475f4cd4827bc2fe77c707 Mon Sep 17 00:00:00 2001 From: Anshum Gupta Date: Mon, 5 Aug 2024 12:37:57 -0700 Subject: [PATCH 19/38] Add branch protection for branch_9_7 --- .asf.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/.asf.yaml b/.asf.yaml index 602feeb246a..57999445552 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -28,6 +28,7 @@ github: branch_9_4: {} branch_9_5: {} branch_9_6: {} + branch_9_7: {} branch_9x: {} protected_tags: From ba1d2e41d3206ce1593c8335a2ca3ec6396c66e5 Mon Sep 17 00:00:00 2001 From: Houston Putman Date: Tue, 6 Aug 2024 15:16:47 -0500 Subject: [PATCH 20/38] SOLR-12963: Fix test failures do to uninvertible change --- .../configsets/doc-expiry/conf/schema.xml | 30 +++++++++---------- .../src/test-files/conf/schema.xml | 4 +-- .../src/test-files/conf/schema.xml | 4 +-- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml index 8a4eb25bba9..7dd56672a61 100644 --- a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml @@ -32,15 +32,15 @@ 1.6: useDocValuesAsStored defaults to true. 1.7: docValues defaults to true, uninvertible defaults to false. --> - - - - + + + + - - - - + + + + @@ -157,7 +157,7 @@ - + @@ -177,16 +177,16 @@ - + - + - + - + @@ -221,10 +221,10 @@ - + - + diff --git a/solr/modules/gcs-repository/src/test-files/conf/schema.xml b/solr/modules/gcs-repository/src/test-files/conf/schema.xml index b539459ad5e..a3a7cc465c2 100644 --- a/solr/modules/gcs-repository/src/test-files/conf/schema.xml +++ b/solr/modules/gcs-repository/src/test-files/conf/schema.xml @@ -17,8 +17,8 @@ --> - - + + diff --git a/solr/modules/s3-repository/src/test-files/conf/schema.xml b/solr/modules/s3-repository/src/test-files/conf/schema.xml index b539459ad5e..a3a7cc465c2 100644 --- a/solr/modules/s3-repository/src/test-files/conf/schema.xml +++ b/solr/modules/s3-repository/src/test-files/conf/schema.xml @@ -17,8 +17,8 @@ --> - - + + From 7ebbcd3493cc5658ad86480906813144125fbc43 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 6 Aug 2024 23:27:23 +0100 Subject: [PATCH 21/38] SOLR-13350, SOLR-17298: use multiThreaded=false default; document multiThreaded parameter; (#2596) Co-authored-by: Gus Heck --- solr/CHANGES.txt | 2 +- .../apache/solr/handler/component/QueryComponent.java | 2 +- .../pages/configuring-solr-xml.adoc | 1 + .../query-guide/pages/common-query-parameters.adoc | 10 ++++++++++ .../org/apache/solr/common/params/CommonParams.java | 3 +++ 5 files changed, 16 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index f7c8fa4d9ae..791094f6b65 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -130,7 +130,7 @@ Other Changes ================== 9.7.0 ================== New Features --------------------- -* SOLR-13350: Multithreaded search execution (Ishan Chattopadhyaya, Mark Miller, Christine Poerschke, David Smiley, noble) +* SOLR-13350, SOLR-17298: Opt-in multithreaded search execution (Ishan Chattopadhyaya, Mark Miller, Christine Poerschke, David Smiley, noble, Gus Heck) * SOLR-17192: Put an UpdateRequestProcessor-enforced soft-limit on the number of fields allowed in a core. The `NumFieldLimitingUpdateRequestProcessorFactory` limit may be adjusted by raising the factory's `maxFields` setting, toggled in and out of "warning-only" mode using the `warnOnly` setting, or disabled entirely diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java index b7c080ec3b5..3c558feffc3 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java @@ -371,7 +371,7 @@ public void process(ResponseBuilder rb) throws IOException { return; } - final boolean multiThreaded = params.getBool("multiThreaded", true); + final boolean multiThreaded = params.getBool(CommonParams.MULTI_THREADED, false); // -1 as flag if not set. long timeAllowed = params.getLong(CommonParams.TIME_ALLOWED, -1L); diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc index 3f865cf4100..fcec13bef61 100644 --- a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc +++ b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc @@ -178,6 +178,7 @@ Specifies the number of threads that will be assigned to replay updates in paral This pool is shared for all cores of the node. The default value is equal to the number of processors. +[#indexSearcherExecutorThreads] `indexSearcherExecutorThreads`:: + [%autowidth,frame=none] diff --git a/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc b/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc index b058483724a..80e92f48289 100644 --- a/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc +++ b/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc @@ -400,6 +400,16 @@ If early termination is used, a `segmentTerminatedEarly` header will be included Similar to using <>, when early segment termination happens values such as `numFound`, xref:faceting.adoc[Facet] counts, and result xref:stats-component.adoc[Stats] may not be accurate for the entire result set. +== multiThreaded Parameter + +[%autowidth,frame=none] +|=== +|Optional |Default: `false` +|=== + +This parameter set to `true` or `false` controls if Solr may use more than one thread to satisfy the request. +A `true` value presently allows the IndexSearcher to search across Lucene's segments in parallel, and the xref:configuration-guide:configuring-solr-xml.adoc#indexSearcherExecutorThreads[indexSearcherExecutorThreads] value can be customised in the `solr.xml` file. This parameter is ignored in the presence of `&segmentsTerminateEarly=true` (future work may enable it). This is a new parameter and is considered experimental and subject to change or removal in subsequent releases. Please share your feedback and experiences with it on our mailing lists. + == omitHeader Parameter [%autowidth,frame=none] diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java index 9fca76238b3..4e94b1d97c6 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java @@ -167,6 +167,9 @@ public interface CommonParams { /** Timeout value in milliseconds. If not set, or the value is < 0, there is no timeout. */ String TIME_ALLOWED = "timeAllowed"; + /** Whether or not the search may use the multi-threaded logic */ + String MULTI_THREADED = "multiThreaded"; + /** * Maximum query CPU usage value in milliseconds. If not set, or the value is < 0, there is no * timeout. From 7ccf4d3c9af7a03dc5ffce8cceda11c360fbccd0 Mon Sep 17 00:00:00 2001 From: David Smiley Date: Tue, 6 Aug 2024 23:50:06 -0400 Subject: [PATCH 22/38] SOLR-17102: Replaced VersionBucket array with locks on-demand (#2548) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The VersionBucket indexing lock mechanism was replaced with something just as fast yet that which consumes almost no memory, saving 1MB of memory per SolrCore. Removed numVersionBuckets and versionBucketLockTimeoutMs. Refactorings: DistributedUpdateProcessor: some refactoring to balance locks clearly. 
VersionInfo: moved locks out to new UpdateLocks. OrderedExecutor uses generics now. Allows use of the ID directly (a BytesRef) instead of a hash. --- solr/CHANGES.txt | 3 + .../org/apache/solr/core/CoreContainer.java | 7 +- .../solr/update/TimedVersionBucket.java | 83 --- .../org/apache/solr/update/UpdateLocks.java | 192 ++++++ .../org/apache/solr/update/UpdateLog.java | 90 ++- .../org/apache/solr/update/VersionBucket.java | 64 -- .../org/apache/solr/update/VersionInfo.java | 86 +-- .../processor/DistributedUpdateProcessor.java | 569 ++++++++---------- .../org/apache/solr/util/OrderedExecutor.java | 6 +- .../configsets/_default/conf/solrconfig.xml | 1 - .../DistributedUpdateProcessorTest.java | 88 +-- .../apache/solr/util/OrderedExecutorTest.java | 20 +- .../solr/collection1/conf/solrconfig.xml | 1 - .../configsets/_default/conf/solrconfig.xml | 7 - .../conf/solrconfig.xml | 7 - .../pages/commits-transaction-logs.adoc | 34 -- .../pages/major-changes-in-solr-9.adoc | 4 + 17 files changed, 580 insertions(+), 682 deletions(-) delete mode 100644 solr/core/src/java/org/apache/solr/update/TimedVersionBucket.java create mode 100644 solr/core/src/java/org/apache/solr/update/UpdateLocks.java delete mode 100644 solr/core/src/java/org/apache/solr/update/VersionBucket.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 791094f6b65..cdedd6d51a3 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -115,6 +115,9 @@ Optimizations beginning?) in which its collection state cache was not being used, resulting in many extra requests to Solr for cluster information. (Aparna Suresh, shalin, David Smiley) +* SOLR-17102: The VersionBucket indexing lock mechanism was replaced with something just as fast yet + that which consumes almost no memory, saving 1MB of memory per SolrCore. (David Smiley) + Bug Fixes --------------------- (No changes) diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index ed0b2a48654..86c75c7ed77 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -63,6 +63,7 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; import org.apache.solr.api.ClusterPluginsSource; import org.apache.solr.api.ContainerPluginsRegistry; import org.apache.solr.api.JerseyResource; @@ -239,7 +240,7 @@ public JerseyAppHandlerCache getJerseyAppHandlerCache() { ExecutorUtil.newMDCAwareCachedThreadPool( new SolrNamedThreadFactory("coreContainerWorkExecutor")); - private final OrderedExecutor replayUpdatesExecutor; + private final OrderedExecutor replayUpdatesExecutor; protected volatile LogWatcher logging = null; @@ -420,7 +421,7 @@ public CoreContainer(NodeConfig config, CoresLocator locator, boolean asyncSolrC this.containerProperties = new Properties(config.getSolrProperties()); this.asyncSolrCoreLoad = asyncSolrCoreLoad; this.replayUpdatesExecutor = - new OrderedExecutor( + new OrderedExecutor<>( cfg.getReplayUpdatesThreads(), ExecutorUtil.newMDCAwareCachedThreadPool( cfg.getReplayUpdatesThreads(), // thread count @@ -727,7 +728,7 @@ public Tracer getTracer() { return tracer; } - public OrderedExecutor getReplayUpdatesExecutor() { + public OrderedExecutor getReplayUpdatesExecutor() { return replayUpdatesExecutor; } diff --git a/solr/core/src/java/org/apache/solr/update/TimedVersionBucket.java b/solr/core/src/java/org/apache/solr/update/TimedVersionBucket.java deleted file mode 100644 index cdddede5a79..00000000000 --- a/solr/core/src/java/org/apache/solr/update/TimedVersionBucket.java +++ /dev/null @@ -1,83 +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.update; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import org.apache.solr.common.SolrException; - -/** - * This implementation uses lock and condition and will throw exception if it can't obtain the lock - * within lockTimeoutMs. - * - * @lucene.internal - */ -public class TimedVersionBucket extends VersionBucket { - - private final Lock lock = new ReentrantLock(true); - private final Condition condition = lock.newCondition(); - - /** - * This will run the function with the lock. It will throw exception if it can't obtain the lock - * within lockTimeoutMs. - */ - @Override - public R runWithLock(int lockTimeoutMs, CheckedFunction function) - throws IOException { - if (tryLock(lockTimeoutMs)) { - return function.apply(); - } else { - throw new SolrException( - SolrException.ErrorCode.SERVER_ERROR, - "Unable to get version bucket lock in " + lockTimeoutMs + " ms"); - } - } - - @Override - public void unlock() { - lock.unlock(); - } - - @Override - public void signalAll() { - condition.signalAll(); - } - - @Override - public void awaitNanos(long nanosTimeout) { - try { - if (nanosTimeout > 0) { - condition.awaitNanos(nanosTimeout); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - protected boolean tryLock(int lockTimeoutMs) { - try { - return lock.tryLock(lockTimeoutMs, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } -} diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLocks.java b/solr/core/src/java/org/apache/solr/update/UpdateLocks.java new file mode 100644 index 00000000000..5fbd27c422c --- /dev/null +++ b/solr/core/src/java/org/apache/solr/update/UpdateLocks.java @@ -0,0 +1,192 @@ +/* + * 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 com.carrotsearch.hppc.IntObjectHashMap; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.lucene.util.BytesRef; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.util.IOFunction; + +/** + * Locks associated with updates in connection with the {@link UpdateLog}. + * + * @lucene.internal + */ +public class UpdateLocks { + + private final long docLockTimeoutMs; + + private final ReadWriteLock blockUpdatesLock = new ReentrantReadWriteLock(true); + + // SolrCloud's first approach was a fixed size array of locks (lock striping) using an ID's hash. + // Sized too small, there was too much lock sharing; sized too big, there was memory waste. + // Here we have a Map keyed by hash and a pool of locks to re-use. Synchronization is needed 2x. + // Note: ConcurrentHashMap was also explored but HPPC came out on top, probably because + // we can use a hashcode directly as the key, and it's GC friendly (zero-allocation). + + /** Maps a ID hashcode to a lock. Synchronize to manipulate. */ + private final IntObjectHashMap hashToLock = + new IntObjectHashMap<>(32) { + @Override + protected int hashKey(int key) { + return key; // our keys are themselves hash-codes + } + }; + + /** A pool of locks to avoid creating & GC'ing them too much. Must synchronize on hashToLock. */ + private final ArrayDeque lockPool = new ArrayDeque<>(16); + + public UpdateLocks(long docLockTimeoutMs) { + this.docLockTimeoutMs = docLockTimeoutMs; + } + + /** + * Acquires a lock for the given doc ID, executes the function, and releases the lock. The + * provided {@link Condition} can be used for wait/notify control. The per-doc locking is needed + * indirectly due to SolrCloud's internal out-of-order versioned update design. + */ + public R runWithLock(BytesRef id, IOFunction function) throws IOException { + final var startTimeNanos = System.nanoTime(); + + lockForUpdate(); + try { + // note: if we didn't need a Condition, then we could have reused + // OrderedExecutor.SparseStripedLock over here, which is also a mechanism invented for + // per-doc locking. + + // hashToLock isn't concurrent, but we synchronize on it briefly twice to do cheap work + + final int hash = id.hashCode(); + final LockAndCondition lock; + // get or insert lock, increment refcount + synchronized (hashToLock) { + final int idx = hashToLock.indexOf(hash); + if (hashToLock.indexExists(idx)) { + lock = hashToLock.indexGet(idx); + assert lock.refCount >= 1; + lock.refCount++; + } else { + lock = borrowLock(); + hashToLock.indexInsert(idx, hash, lock); + } + } + + // try-finally ensuring we decrement the refCount + try { + return runWithLockInternal(id, function, lock, startTimeNanos); + } finally { + // decrement refcount, remove lock if unreferenced + synchronized (hashToLock) { + assert lock.refCount > 0; // because we incremented it + if (--lock.refCount == 0) { // typical + hashToLock.remove(hash); + returnLock(lock); + } + } + } + + } finally { + unlockForUpdate(); + } + } + + private LockAndCondition borrowLock() { + assert Thread.holdsLock(hashToLock); + if (lockPool.isEmpty()) { + return new LockAndCondition(); + } else { + return lockPool.removeLast(); + } + } + + private void returnLock(LockAndCondition lock) { + assert Thread.holdsLock(hashToLock); + if (lockPool.size() < 16) { + lockPool.add(lock); + lock.refCount = 1; // ready for next use + } + } + + private R runWithLockInternal( + BytesRef id, IOFunction function, LockAndCondition lock, long startTimeNanos) + throws IOException { + // Acquire the lock + try { + if (docLockTimeoutMs == 0) { + lock.lock.lockInterruptibly(); + } else { + long remainingNs = + TimeUnit.MILLISECONDS.toNanos(docLockTimeoutMs) - (System.nanoTime() - startTimeNanos); + boolean timedOut = !lock.lock.tryLock(remainingNs, TimeUnit.NANOSECONDS); + if (timedOut) { + throw new SolrException( + ErrorCode.SERVER_ERROR, + "Unable to lock doc " + id + " in " + docLockTimeoutMs + " ms"); + } + } + } catch (InterruptedException e) { + // don't set interrupt status; we're ending the request + throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to lock doc " + id, e); + } + // try-finally ensuring we unlock + try { + // We have the lock; do stuff with it + return function.apply(lock.condition); + } finally { + // Release the lock + lock.lock.unlock(); + } + } + + private static class LockAndCondition { + final Lock lock; + final Condition condition; + int refCount; // only access when synchronized on hashToLock + + LockAndCondition() { + lock = new ReentrantLock(true); // fair + condition = lock.newCondition(); + refCount = 1; + } + } + + public void lockForUpdate() { + blockUpdatesLock.readLock().lock(); + } + + public void unlockForUpdate() { + blockUpdatesLock.readLock().unlock(); + } + + public void blockUpdates() { + blockUpdatesLock.writeLock().lock(); + } + + public void unblockUpdates() { + blockUpdatesLock.writeLock().unlock(); + } +} diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java index 4ebe4f311d1..9bc685f276b 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -66,6 +66,7 @@ import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.CollectionUtil; +import org.apache.solr.common.util.EnvUtils; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.SolrNamedThreadFactory; @@ -223,15 +224,14 @@ public String toString() { protected final int numDeletesByQueryToKeep = 100; protected int numRecordsToKeep; protected int maxNumLogsToKeep; - // This should only be used to initialize VersionInfo... the actual number of buckets may be - // rounded up to a power of two. - protected int numVersionBuckets; protected boolean existOldBufferLog = false; // keep track of deletes only... this is not updated on an add protected LinkedHashMap oldDeletes = new OldDeletesLinkedHashMap(this.numDeletesToKeep); + private UpdateLocks updateLocks; + /** Holds the query and the version for a DeleteByQuery command */ public static class DBQ { public String q; // the query string @@ -331,6 +331,10 @@ public VersionInfo getVersionInfo() { return versionInfo; } + public UpdateLocks getLocks() { + return updateLocks; + } + public int getNumRecordsToKeep() { return numRecordsToKeep; } @@ -339,10 +343,6 @@ public int getMaxNumLogsToKeep() { return maxNumLogsToKeep; } - public int getNumVersionBuckets() { - return numVersionBuckets; - } - protected static int objToInt(Object obj, int def) { if (obj != null) { return Integer.parseInt(obj.toString()); @@ -381,19 +381,25 @@ public void init(PluginInfo info) { numRecordsToKeep = objToInt(info.initArgs.get("numRecordsToKeep"), 100); maxNumLogsToKeep = objToInt(info.initArgs.get("maxNumLogsToKeep"), 10); - numVersionBuckets = objToInt(info.initArgs.get("numVersionBuckets"), 65536); - if (numVersionBuckets <= 0) - throw new SolrException( - SolrException.ErrorCode.SERVER_ERROR, - "Number of version buckets must be greater than 0!"); + if (info.initArgs.get("numVersionBuckets") != null) { + log.warn("numVersionBuckets is obsolete"); + } + + if (info.initArgs.get("versionBucketLockTimeoutMs") != null) { + log.warn("versionBucketLockTimeoutMs is mostly replaced by docLockTimeoutMs"); + } + int timeoutMs = + objToInt( + info.initArgs.get("docLockTimeoutMs", info.initArgs.get("versionBucketLockTimeoutMs")), + EnvUtils.getPropertyAsLong("solr.update.docLockTimeoutMs", 0L).intValue()); + updateLocks = new UpdateLocks(timeoutMs); log.info( - "Initializing UpdateLog: dataDir={} defaultSyncLevel={} numRecordsToKeep={} maxNumLogsToKeep={} numVersionBuckets={}", + "Initializing UpdateLog: dataDir={} defaultSyncLevel={} numRecordsToKeep={} maxNumLogsToKeep={}", dataDir, defaultSyncLevel, numRecordsToKeep, - maxNumLogsToKeep, - numVersionBuckets); + maxNumLogsToKeep); } private final AtomicBoolean initialized = new AtomicBoolean(); @@ -443,7 +449,6 @@ public void init(UpdateHandler uhandler, SolrCore core) { getTlogDir(), id); } - versionInfo.reload(); core.getCoreMetricManager() .registerMetricProducer(SolrInfoBean.Category.TLOG.toString(), this); @@ -464,7 +469,7 @@ public void init(UpdateHandler uhandler, SolrCore core) { usableForChildDocs = core.getLatestSchema().isUsableForChildDocs(); try { - versionInfo = new VersionInfo(this, numVersionBuckets); + versionInfo = new VersionInfo(this); } catch (SolrException e) { log.error("Unable to use updateLog: ", e); throw new SolrException( @@ -1373,7 +1378,7 @@ public Future recoverFromLog() { ExecutorCompletionService cs = new ExecutorCompletionService<>(recoveryExecutor); LogReplayer replayer = new LogReplayer(recoverLogs, false); - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { state = State.REPLAYING; @@ -1383,7 +1388,7 @@ public Future recoverFromLog() { deleteByQueries.clear(); oldDeletes.clear(); } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } // At this point, we are guaranteed that any new updates coming in will see the state as @@ -1409,11 +1414,11 @@ public Future recoverFromCurrentLog() { ExecutorCompletionService cs = new ExecutorCompletionService<>(recoveryExecutor); LogReplayer replayer = new LogReplayer(Collections.singletonList(tlog), false, true); - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { state = State.REPLAYING; } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } return cs.submit(replayer, recoveryInfo); @@ -1427,7 +1432,7 @@ public Future recoverFromCurrentLog() { * @param cuc any updates that have version larger than the version of cuc will be copied over */ public void copyOverBufferingUpdates(CommitUpdateCommand cuc) { - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { synchronized (this) { state = State.ACTIVE; @@ -1440,7 +1445,7 @@ public void copyOverBufferingUpdates(CommitUpdateCommand cuc) { dropBufferTlog(); } } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } } @@ -1451,7 +1456,7 @@ public void copyOverBufferingUpdates(CommitUpdateCommand cuc) { * @param cuc any updates that have version larger than the version of cuc will be copied over */ public void commitAndSwitchToNewTlog(CommitUpdateCommand cuc) { - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { synchronized (this) { if (tlog == null) { @@ -1465,7 +1470,7 @@ public void commitAndSwitchToNewTlog(CommitUpdateCommand cuc) { } } } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } } @@ -1888,7 +1893,7 @@ public void bufferUpdates() { // block all updates to eliminate race conditions // reading state and acting on it in the distributed update processor - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { if (state != State.ACTIVE && state != State.BUFFERING) { // we don't currently have support for handling other states @@ -1906,13 +1911,13 @@ public void bufferUpdates() { state = State.BUFFERING; } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } } /** Returns true if we were able to drop buffered updates and return to the ACTIVE state */ public boolean dropBufferedUpdates() { - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { if (state != State.BUFFERING) return false; @@ -1924,7 +1929,7 @@ public boolean dropBufferedUpdates() { state = State.ACTIVE; } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } return true; } @@ -1946,7 +1951,7 @@ public Future applyBufferedUpdates() { // block all updates to eliminate race conditions // reading state and acting on it in the update processor - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); try { cancelApplyBufferUpdate = false; if (state != State.BUFFERING) return null; @@ -1962,7 +1967,7 @@ public Future applyBufferedUpdates() { state = State.APPLYING_BUFFERED; } finally { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } if (ExecutorUtil.isShutdown(recoveryExecutor)) { @@ -2048,7 +2053,7 @@ public void run() { // change the state while updates are still blocked to prevent races state = State.ACTIVE; if (finishing) { - versionInfo.unblockUpdates(); + updateLocks.unblockUpdates(); } // clean up in case we hit some unexpected exception and didn't get @@ -2100,7 +2105,7 @@ public void doReplay(TransactionLog translog) { return proc; }); - OrderedExecutor executor = + OrderedExecutor executor = inSortedOrder ? null : req.getCoreContainer().getReplayUpdatesExecutor(); AtomicInteger pendingTasks = new AtomicInteger(0); AtomicReference exceptionOnExecuteUpdate = new AtomicReference<>(); @@ -2147,7 +2152,7 @@ public void doReplay(TransactionLog translog) { // after we've finished this recovery. // NOTE: our own updates won't be blocked since the thread holding a write lock can // lock a read lock. - versionInfo.blockUpdates(); + updateLocks.blockUpdates(); finishing = true; o = tlogReader.next(); } else { @@ -2309,34 +2314,27 @@ private void waitForAllUpdatesGetExecuted(AtomicInteger pendingTasks) { } } - private Integer getBucketHash(UpdateCommand cmd) { + private BytesRef getIndexedId(UpdateCommand cmd) { if (cmd instanceof AddUpdateCommand) { - BytesRef idBytes = ((AddUpdateCommand) cmd).getIndexedId(); - if (idBytes == null) return null; - return DistributedUpdateProcessor.bucketHash(idBytes); + return ((AddUpdateCommand) cmd).getIndexedId(); } - if (cmd instanceof DeleteUpdateCommand) { - BytesRef idBytes = ((DeleteUpdateCommand) cmd).getIndexedId(); - if (idBytes == null) return null; - return DistributedUpdateProcessor.bucketHash(idBytes); + return ((DeleteUpdateCommand) cmd).getIndexedId(); } - return null; } private void execute( UpdateCommand cmd, - OrderedExecutor executor, + OrderedExecutor executor, AtomicInteger pendingTasks, ThreadLocal procTl, AtomicReference exceptionHolder) { assert cmd instanceof AddUpdateCommand || cmd instanceof DeleteUpdateCommand; if (executor != null) { - // by using the same hash as DUP, independent updates can avoid waiting for same bucket executor.execute( - getBucketHash(cmd), + getIndexedId(cmd), () -> { try { // fail fast diff --git a/solr/core/src/java/org/apache/solr/update/VersionBucket.java b/solr/core/src/java/org/apache/solr/update/VersionBucket.java deleted file mode 100644 index 7724b3a2cb4..00000000000 --- a/solr/core/src/java/org/apache/solr/update/VersionBucket.java +++ /dev/null @@ -1,64 +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.update; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; - -// TODO: make inner? -// TODO: store the highest possible in the index on a commit (but how to not block adds?) -// TODO: could also store highest possible in the transaction log after a commit. -// Or on a new index, just scan "version" for the max? -/** - * The default implementation which uses the intrinsic object monitor. It uses less memory but - * ignores the lockTimeoutMs. - * - * @lucene.internal - */ -public class VersionBucket { - - @FunctionalInterface - public interface CheckedFunction { - R apply() throws IOException; - } - - /** This will run the function with the intrinsic object monitor. */ - public R runWithLock(int lockTimeoutMs, CheckedFunction function) - throws IOException { - synchronized (this) { - return function.apply(); - } - } - - /** Nothing to do for the intrinsic object monitor. */ - public void unlock() {} - - public void signalAll() { - notifyAll(); - } - - public void awaitNanos(long nanosTimeout) { - try { - long millis = TimeUnit.NANOSECONDS.toMillis(nanosTimeout); - if (millis > 0) { - wait(millis); - } - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } -} diff --git a/solr/core/src/java/org/apache/solr/update/VersionInfo.java b/solr/core/src/java/org/apache/solr/update/VersionInfo.java index 09a39397197..ad7c427c403 100644 --- a/solr/core/src/java/org/apache/solr/update/VersionInfo.java +++ b/solr/core/src/java/org/apache/solr/update/VersionInfo.java @@ -20,11 +20,8 @@ import java.io.IOException; import java.util.Map; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.lucene.queries.function.FunctionValues; import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.solr.common.SolrException; import org.apache.solr.common.util.SuppressForbidden; @@ -33,18 +30,16 @@ import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.RefCounted; +/** + * Related to the {@code _version_} field, in connection with the {@link UpdateLog}. + * + * @lucene.internal + */ public class VersionInfo { - private static final String SYS_PROP_BUCKET_VERSION_LOCK_TIMEOUT_MS = - "bucketVersionLockTimeoutMs"; private final UpdateLog ulog; - private final int numBuckets; - private volatile VersionBucket[] buckets; - private final Object bucketsSync = new Object(); - private final SchemaField versionField; - final ReadWriteLock lock = new ReentrantReadWriteLock(true); - private final int versionBucketLockTimeoutMs; + private final SchemaField versionField; /** * Gets and returns the {@link org.apache.solr.common.params.CommonParams#VERSION_FIELD} from the @@ -82,47 +77,16 @@ public static SchemaField getAndCheckVersionField(IndexSchema schema) throws Sol return sf; } - public VersionInfo(UpdateLog ulog, int nBuckets) { + public VersionInfo(UpdateLog ulog) { this.ulog = ulog; IndexSchema schema = ulog.uhandler.core.getLatestSchema(); versionField = getAndCheckVersionField(schema); - versionBucketLockTimeoutMs = - ulog.uhandler - .core - .getSolrConfig() - .get("updateHandler") - .get("versionBucketLockTimeoutMs") - .intVal( - Integer.parseInt(System.getProperty(SYS_PROP_BUCKET_VERSION_LOCK_TIMEOUT_MS, "0"))); - numBuckets = BitUtil.nextHighestPowerOfTwo(nBuckets); } - public int getVersionBucketLockTimeoutMs() { - return versionBucketLockTimeoutMs; - } - - public void reload() {} - public SchemaField getVersionField() { return versionField; } - public void lockForUpdate() { - lock.readLock().lock(); - } - - public void unlockForUpdate() { - lock.readLock().unlock(); - } - - public void blockUpdates() { - lock.writeLock().lock(); - } - - public void unblockUpdates() { - lock.writeLock().unlock(); - } - /* // todo: initialize... use current time to start? // a clock that increments by 1 for every operation makes it easier to detect missing @@ -172,42 +136,6 @@ public long getNewClock() { } } - public long getOldClock() { - synchronized (clockSync) { - return vclock; - } - } - - public void updateClock(long clock) { - synchronized (clockSync) { - vclock = Math.max(vclock, clock); - } - } - - public VersionBucket bucket(int hash) { - // If this is a user provided hash, it may be poor in the right-hand bits. - // Make sure high bits are moved down, since only the low bits will matter. - // int h = hash + (hash >>> 8) + (hash >>> 16) + (hash >>> 24); - // Assume good hash codes for now. - int slot = hash & (numBuckets - 1); - if (buckets == null) { - synchronized (bucketsSync) { - if (buckets == null) { - buckets = createVersionBuckets(); - } - } - } - return buckets[slot]; - } - - private VersionBucket[] createVersionBuckets() { - VersionBucket[] buckets = new VersionBucket[numBuckets]; - for (int i = 0; i < buckets.length; i++) { - buckets[i] = versionBucketLockTimeoutMs > 0 ? new TimedVersionBucket() : new VersionBucket(); - } - return buckets; - } - public Long lookupVersion(BytesRef idBytes) { return ulog.lookupVersion(idBytes); } diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java index 8588a0a34de..d8c88ac5786 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CharsRefBuilder; import org.apache.solr.client.solrj.SolrRequest; @@ -43,7 +44,6 @@ import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.UpdateParams; -import org.apache.solr.common.util.Hash; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.TimeSource; import org.apache.solr.handler.component.RealTimeGetComponent; @@ -57,9 +57,9 @@ import org.apache.solr.update.SolrCmdDistributor.Node; import org.apache.solr.update.SolrCmdDistributor.SolrError; import org.apache.solr.update.UpdateCommand; +import org.apache.solr.update.UpdateLocks; import org.apache.solr.update.UpdateLog; import org.apache.solr.update.UpdateShardHandler; -import org.apache.solr.update.VersionBucket; import org.apache.solr.update.VersionInfo; import org.apache.solr.util.TestInjection; import org.apache.solr.util.TimeOut; @@ -129,7 +129,7 @@ public static DistribPhase parseParam(final String param) { private final AtomicUpdateDocumentMerger docMerger; private final UpdateLog ulog; - @VisibleForTesting VersionInfo vinfo; + private final VersionInfo vinfo; private final boolean versionsStored; private boolean returnVersions; @@ -202,6 +202,11 @@ public DistributedUpdateProcessor( // this.rsp = reqInfo != null ? reqInfo.getRsp() : null; } + @VisibleForTesting + protected UpdateLocks getUpdateLocks() { + return ulog.getLocks(); + } + /** * @return the replica type of the collection. */ @@ -277,11 +282,6 @@ private void doLocalDelete(DeleteUpdateCommand cmd) throws IOException { isIndexChanged = true; } - public static int bucketHash(BytesRef idBytes) { - assert idBytes != null; - return Hash.murmurhash3_x86_32(idBytes.bytes, idBytes.offset, idBytes.length, 0); - } - /** * @return whether or not to drop this cmd * @throws IOException If there is a low-level I/O error. @@ -305,10 +305,6 @@ protected boolean versionAdd(AddUpdateCommand cmd) throws IOException { } } - // This is only the hash for the bucket, and must be based only on the uniqueKey (i.e. do not - // use a pluggable hash here) - int bucketHash = bucketHash(idBytes); - // at this point, there is an update we need to try and apply. // we may or may not be the leader. @@ -335,36 +331,41 @@ protected boolean versionAdd(AddUpdateCommand cmd) throws IOException { leaderLogicWithVersionIntegrityCheck(isReplayOrPeersync, isLeader, versionOnUpdate); boolean forwardedFromCollection = cmd.getReq().getParams().get(DISTRIB_FROM_COLLECTION) != null; - VersionBucket bucket = vinfo.bucket(bucketHash); - long dependentVersionFound = -1; // if this is an in-place update, check and wait if we should be waiting for a previous update // (on which this update depends), before entering the synchronized block if (!leaderLogic && cmd.isInPlaceUpdate()) { - dependentVersionFound = - waitForDependentUpdates(cmd, versionOnUpdate, isReplayOrPeersync, bucket); + dependentVersionFound = waitForDependentUpdates(cmd, versionOnUpdate, isReplayOrPeersync); if (dependentVersionFound == -1) { // it means the document has been deleted by now at the leader. drop this update return true; } } - vinfo.lockForUpdate(); - try { - long finalVersionOnUpdate = versionOnUpdate; - return bucket.runWithLock( - vinfo.getVersionBucketLockTimeoutMs(), - () -> - doVersionAdd( + final long finalVersionOnUpdate = versionOnUpdate; + return getUpdateLocks() + .runWithLock( + cmd.getIndexedId(), + (Condition condition) -> { + // just in case anyone is waiting let them know that we have a new update + // we obtain the version when synchronized and then do the add so we can ensure that + // if version1 < version2 then version1 is actually added before version2. + + // even if we don't store the version field, synchronizing + // will enable us to know what version happened first, and thus enable + // realtime-get to work reliably. + // TODO: if versions aren't stored, do we need to set on the cmd anyway for some + // reason? + // there may be other reasons in the future for a version on the commands + condition.signalAll(); + + return doVersionAdd( cmd, finalVersionOnUpdate, isReplayOrPeersync, leaderLogic, - forwardedFromCollection, - bucket)); - } finally { - vinfo.unlockForUpdate(); - } + forwardedFromCollection); + }); } private boolean doVersionAdd( @@ -372,172 +373,157 @@ private boolean doVersionAdd( long versionOnUpdate, boolean isReplayOrPeersync, boolean leaderLogic, - boolean forwardedFromCollection, - VersionBucket bucket) + boolean forwardedFromCollection) throws IOException { - try { - BytesRef idBytes = cmd.getIndexedId(); - bucket.signalAll(); - // just in case anyone is waiting let them know that we have a new update - // we obtain the version when synchronized and then do the add so we can ensure that - // if version1 < version2 then version1 is actually added before version2. - - // even if we don't store the version field, synchronizing on the bucket - // will enable us to know what version happened first, and thus enable - // realtime-get to work reliably. - // TODO: if versions aren't stored, do we need to set on the cmd anyway for some reason? - // there may be other reasons in the future for a version on the commands - - if (versionsStored) { - - if (leaderLogic) { - - if (forwardedFromCollection && ulog.getState() == UpdateLog.State.ACTIVE) { - // forwarded from a collection but we are not buffering so strip original version and - // apply our own - // see SOLR-5308 - if (log.isInfoEnabled()) { - log.info("Removing version field from doc: {}", cmd.getPrintableId()); - } - cmd.solrDoc.remove(CommonParams.VERSION_FIELD); - versionOnUpdate = 0; - } + BytesRef idBytes = cmd.getIndexedId(); - getUpdatedDocument(cmd, versionOnUpdate); + if (versionsStored) { - // leaders can also be in buffering state during "migrate" API call, see SOLR-5308 - if (forwardedFromCollection - && ulog.getState() != UpdateLog.State.ACTIVE - && isReplayOrPeersync == false) { - // we're not in an active state, and this update isn't from a replay, so buffer it. - if (log.isInfoEnabled()) { - log.info( - "Leader logic applied but update log is buffering: {}", cmd.getPrintableId()); - } - cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); - ulog.add(cmd); - return true; + if (leaderLogic) { + + if (forwardedFromCollection && ulog.getState() == UpdateLog.State.ACTIVE) { + // forwarded from a collection but we are not buffering so strip original version and + // apply our own + // see SOLR-5308 + if (log.isInfoEnabled()) { + log.info("Removing version field from doc: {}", cmd.getPrintableId()); } + cmd.solrDoc.remove(CommonParams.VERSION_FIELD); + versionOnUpdate = 0; + } - if (versionOnUpdate != 0) { - Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); - long foundVersion = lastVersion == null ? -1 : lastVersion; - if (versionOnUpdate == foundVersion - || (versionOnUpdate < 0 && foundVersion < 0) - || (versionOnUpdate == 1 && foundVersion > 0)) { - // we're ok if versions match, or if both are negative (all missing docs are equal), - // or if cmd specified it must exist (versionOnUpdate==1) and it does. - } else { - if (cmd.getReq().getParams().getBool(CommonParams.FAIL_ON_VERSION_CONFLICTS, true) - == false) { - return true; - } + getUpdatedDocument(cmd, versionOnUpdate); - throw new SolrException( - ErrorCode.CONFLICT, - "version conflict for " - + cmd.getPrintableId() - + " expected=" - + versionOnUpdate - + " actual=" - + foundVersion); - } + // leaders can also be in buffering state during "migrate" API call, see SOLR-5308 + if (forwardedFromCollection + && ulog.getState() != UpdateLog.State.ACTIVE + && isReplayOrPeersync == false) { + // we're not in an active state, and this update isn't from a replay, so buffer it. + if (log.isInfoEnabled()) { + log.info("Leader logic applied but update log is buffering: {}", cmd.getPrintableId()); } + cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); + ulog.add(cmd); + return true; + } - long version = vinfo.getNewClock(); - cmd.setVersion(version); - cmd.getSolrInputDocument().setField(CommonParams.VERSION_FIELD, version); - } else { - // The leader forwarded us this update. - cmd.setVersion(versionOnUpdate); + if (versionOnUpdate != 0) { + Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); + long foundVersion = lastVersion == null ? -1 : lastVersion; + if (versionOnUpdate == foundVersion + || (versionOnUpdate < 0 && foundVersion < 0) + || (versionOnUpdate == 1 && foundVersion > 0)) { + // we're ok if versions match, or if both are negative (all missing docs are equal), + // or if cmd specified it must exist (versionOnUpdate==1) and it does. + } else { + if (cmd.getReq().getParams().getBool(CommonParams.FAIL_ON_VERSION_CONFLICTS, true) + == false) { + return true; + } - if (shouldBufferUpdate(cmd, isReplayOrPeersync, ulog.getState())) { - // we're not in an active state, and this update isn't from a replay, so buffer it. - cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); - ulog.add(cmd); - return true; + throw new SolrException( + ErrorCode.CONFLICT, + "version conflict for " + + cmd.getPrintableId() + + " expected=" + + versionOnUpdate + + " actual=" + + foundVersion); } + } - if (cmd.isInPlaceUpdate()) { - long prev = cmd.prevVersion; - Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); - if (lastVersion == null || Math.abs(lastVersion) < prev) { - // this was checked for (in waitForDependentUpdates()) before entering the - // synchronized block. So we shouldn't be here, unless what must've happened is: by - // the time synchronization block was entered, the prev update was deleted by DBQ. - // Since now that update is not in index, the vinfo.lookupVersion() is possibly giving - // us a version from the deleted list (which might be older than the prev update!) - UpdateCommand fetchedFromLeader = fetchFullUpdateFromLeader(cmd, versionOnUpdate); - - if (fetchedFromLeader instanceof DeleteUpdateCommand) { - if (log.isInfoEnabled()) { - log.info( - "In-place update of {} failed to find valid lastVersion to apply to, and the document was deleted at the leader subsequently.", - idBytes.utf8ToString()); - } - versionDelete((DeleteUpdateCommand) fetchedFromLeader); - return true; - } else { - assert fetchedFromLeader instanceof AddUpdateCommand; - // Newer document was fetched from the leader. Apply that document instead of this - // current in-place update. - if (log.isInfoEnabled()) { - log.info( - "In-place update of {} failed to find valid lastVersion to apply to, forced to fetch full doc from leader: {}", - idBytes.utf8ToString(), - fetchedFromLeader); - } - // Make this update to become a non-inplace update containing the full document - // obtained from the leader - cmd.solrDoc = ((AddUpdateCommand) fetchedFromLeader).solrDoc; - cmd.prevVersion = -1; - cmd.setVersion((long) cmd.solrDoc.getFieldValue(CommonParams.VERSION_FIELD)); - assert cmd.isInPlaceUpdate() == false; + long version = vinfo.getNewClock(); + cmd.setVersion(version); + cmd.getSolrInputDocument().setField(CommonParams.VERSION_FIELD, version); + } else { + // The leader forwarded us this update. + cmd.setVersion(versionOnUpdate); + + if (shouldBufferUpdate(cmd, isReplayOrPeersync, ulog.getState())) { + // we're not in an active state, and this update isn't from a replay, so buffer it. + cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); + ulog.add(cmd); + return true; + } + + if (cmd.isInPlaceUpdate()) { + long prev = cmd.prevVersion; + Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); + if (lastVersion == null || Math.abs(lastVersion) < prev) { + // this was checked for (in waitForDependentUpdates()) before entering the + // synchronized block. So we shouldn't be here, unless what must've happened is: by + // the time synchronization block was entered, the prev update was deleted by DBQ. + // Since now that update is not in index, the vinfo.lookupVersion() is possibly giving + // us a version from the deleted list (which might be older than the prev update!) + UpdateCommand fetchedFromLeader = fetchFullUpdateFromLeader(cmd, versionOnUpdate); + + if (fetchedFromLeader instanceof DeleteUpdateCommand) { + if (log.isInfoEnabled()) { + log.info( + "In-place update of {} failed to find valid lastVersion to apply to, and the document was deleted at the leader subsequently.", + idBytes.utf8ToString()); } + versionDelete((DeleteUpdateCommand) fetchedFromLeader); + return true; } else { - if (Math.abs(lastVersion) > prev) { - // this means we got a newer full doc update and in that case it makes no sense to - // apply the older inplace update. Drop this update + assert fetchedFromLeader instanceof AddUpdateCommand; + // Newer document was fetched from the leader. Apply that document instead of this + // current in-place update. + if (log.isInfoEnabled()) { log.info( - "Update was applied on version: {}, but last version I have is: {}. Dropping current update", - prev, - lastVersion); - return true; - } else { - // We're good, we should apply this update. + "In-place update of {} failed to find valid lastVersion to apply to, forced to fetch full doc from leader: {}", + idBytes.utf8ToString(), + fetchedFromLeader); } + // Make this update to become a non-inplace update containing the full document + // obtained from the leader + cmd.solrDoc = ((AddUpdateCommand) fetchedFromLeader).solrDoc; + cmd.prevVersion = -1; + cmd.setVersion((long) cmd.solrDoc.getFieldValue(CommonParams.VERSION_FIELD)); + assert cmd.isInPlaceUpdate() == false; } } else { - // if we aren't the leader, then we need to check that updates were not re-ordered - // we need to check the specific version for this id. - Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); - if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) { - // This update is a repeat, or was reordered. We need to drop this update. - if (log.isDebugEnabled()) { - log.debug("Dropping add update due to version {}", idBytes.utf8ToString()); - } + if (Math.abs(lastVersion) > prev) { + // this means we got a newer full doc update and in that case it makes no sense to + // apply the older inplace update. Drop this update + log.info( + "Update was applied on version: {}, but last version I have is: {}. Dropping current update", + prev, + lastVersion); return true; + } else { + // We're good, we should apply this update. } } - if (!isSubShardLeader - && replicaType == Replica.Type.TLOG - && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { - cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); + } else { + // if we aren't the leader, then we need to check that updates were not re-ordered + // we need to check the specific version for this id. + Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); + if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) { + // This update is a repeat, or was reordered. We need to drop this update. + if (log.isDebugEnabled()) { + log.debug("Dropping add update due to version {}", idBytes.utf8ToString()); + } + return true; } } + if (!isSubShardLeader + && replicaType == Replica.Type.TLOG + && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { + cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); + } } + } - SolrInputDocument clonedDoc = shouldCloneCmdDoc() ? cmd.solrDoc.deepCopy() : null; + SolrInputDocument clonedDoc = shouldCloneCmdDoc() ? cmd.solrDoc.deepCopy() : null; - // TODO: possibly set checkDeleteByQueries as a flag on the command? - doLocalAdd(cmd); + // TODO: possibly set checkDeleteByQueries as a flag on the command? + doLocalAdd(cmd); - if (clonedDoc != null) { - cmd.solrDoc = clonedDoc; - } - } finally { - bucket.unlock(); + if (clonedDoc != null) { + cmd.solrDoc = clonedDoc; } + return false; } @@ -573,22 +559,16 @@ boolean shouldBufferUpdate( * has been indexed. */ private long waitForDependentUpdates( - AddUpdateCommand cmd, long versionOnUpdate, boolean isReplayOrPeersync, VersionBucket bucket) - throws IOException { - long lastFoundVersion = 0; + AddUpdateCommand cmd, long versionOnUpdate, boolean isReplayOrPeersync) throws IOException { TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME); - vinfo.lockForUpdate(); - try { - lastFoundVersion = - bucket.runWithLock( - vinfo.getVersionBucketLockTimeoutMs(), - () -> - doWaitForDependentUpdates( - cmd, versionOnUpdate, isReplayOrPeersync, bucket, waitTimeout)); - } finally { - vinfo.unlockForUpdate(); - } + long lastFoundVersion = + getUpdateLocks() + .runWithLock( + cmd.getIndexedId(), + (Condition condition) -> + doWaitForDependentUpdates( + cmd, versionOnUpdate, isReplayOrPeersync, condition, waitTimeout)); if (Math.abs(lastFoundVersion) > cmd.prevVersion) { // This must've been the case due to a higher version full update succeeding concurrently, @@ -652,35 +632,34 @@ private long doWaitForDependentUpdates( AddUpdateCommand cmd, long versionOnUpdate, boolean isReplayOrPeersync, - VersionBucket bucket, + Condition condition, TimeOut waitTimeout) { - long lastFoundVersion; - try { - Long lookedUpVersion = vinfo.lookupVersion(cmd.getIndexedId()); - lastFoundVersion = lookedUpVersion == null ? 0L : lookedUpVersion; + Long lookedUpVersion = vinfo.lookupVersion(cmd.getIndexedId()); + long lastFoundVersion = lookedUpVersion == null ? 0L : lookedUpVersion; - if (Math.abs(lastFoundVersion) < cmd.prevVersion) { - if (log.isDebugEnabled()) { - log.debug( - "Re-ordered inplace update. version={}, prevVersion={}, lastVersion={}, replayOrPeerSync={}, id={}", - (cmd.getVersion() == 0 ? versionOnUpdate : cmd.getVersion()), - cmd.prevVersion, - lastFoundVersion, - isReplayOrPeersync, - cmd.getPrintableId()); - } + if (Math.abs(lastFoundVersion) < cmd.prevVersion) { + if (log.isDebugEnabled()) { + log.debug( + "Re-ordered inplace update. version={}, prevVersion={}, lastVersion={}, replayOrPeerSync={}, id={}", + (cmd.getVersion() == 0 ? versionOnUpdate : cmd.getVersion()), + cmd.prevVersion, + lastFoundVersion, + isReplayOrPeersync, + cmd.getPrintableId()); } + } - while (Math.abs(lastFoundVersion) < cmd.prevVersion && !waitTimeout.hasTimedOut()) { - long timeLeftInNanos = waitTimeout.timeLeft(TimeUnit.NANOSECONDS); - if (timeLeftInNanos > 0) { // 0 means: wait forever until notified, but we don't want that. - bucket.awaitNanos(timeLeftInNanos); + while (Math.abs(lastFoundVersion) < cmd.prevVersion && !waitTimeout.hasTimedOut()) { + long timeLeftInNanos = waitTimeout.timeLeft(TimeUnit.NANOSECONDS); + if (timeLeftInNanos > 0) { // 0 means: wait forever until notified, but we don't want that. + try { + condition.await(timeLeftInNanos, TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); } - lookedUpVersion = vinfo.lookupVersion(cmd.getIndexedId()); - lastFoundVersion = lookedUpVersion == null ? 0L : lookedUpVersion; } - } finally { - bucket.unlock(); + lookedUpVersion = vinfo.lookupVersion(cmd.getIndexedId()); + lastFoundVersion = lookedUpVersion == null ? 0L : lookedUpVersion; } return lastFoundVersion; } @@ -958,7 +937,7 @@ protected void versionDeleteByQuery(DeleteUpdateCommand cmd) throws IOException boolean leaderLogic = leaderLogicWithVersionIntegrityCheck(isReplayOrPeersync, isLeader, versionOnUpdate); - vinfo.blockUpdates(); + getUpdateLocks().blockUpdates(); try { doLocalDeleteByQuery(cmd, versionOnUpdate, isReplayOrPeersync, leaderLogic); @@ -968,7 +947,7 @@ protected void versionDeleteByQuery(DeleteUpdateCommand cmd) throws IOException // IndexReader (so cache misses will see up-to-date data) } finally { - vinfo.unblockUpdates(); + getUpdateLocks().unblockUpdates(); } } @@ -1040,10 +1019,6 @@ protected boolean versionDelete(DeleteUpdateCommand cmd) throws IOException { return false; } - // This is only the hash for the bucket, and must be based only on the uniqueKey (i.e. do not - // use a pluggable hash here) - int bucketHash = bucketHash(idBytes); - // at this point, there is an update we need to try and apply. // we may or may not be the leader. @@ -1062,25 +1037,18 @@ protected boolean versionDelete(DeleteUpdateCommand cmd) throws IOException { leaderLogicWithVersionIntegrityCheck(isReplayOrPeersync, isLeader, versionOnUpdate); boolean forwardedFromCollection = cmd.getReq().getParams().get(DISTRIB_FROM_COLLECTION) != null; - VersionBucket bucket = vinfo.bucket(bucketHash); - - vinfo.lockForUpdate(); - try { - long finalVersionOnUpdate = versionOnUpdate; - return bucket.runWithLock( - vinfo.getVersionBucketLockTimeoutMs(), - () -> - doVersionDelete( - cmd, - finalVersionOnUpdate, - signedVersionOnUpdate, - isReplayOrPeersync, - leaderLogic, - forwardedFromCollection, - bucket)); - } finally { - vinfo.unlockForUpdate(); - } + long finalVersionOnUpdate = versionOnUpdate; + return getUpdateLocks() + .runWithLock( + cmd.getIndexedId(), + (Condition unused) -> + doVersionDelete( + cmd, + finalVersionOnUpdate, + signedVersionOnUpdate, + isReplayOrPeersync, + leaderLogic, + forwardedFromCollection)); } private boolean doVersionDelete( @@ -1089,94 +1057,89 @@ private boolean doVersionDelete( long signedVersionOnUpdate, boolean isReplayOrPeersync, boolean leaderLogic, - boolean forwardedFromCollection, - VersionBucket bucket) + boolean forwardedFromCollection) throws IOException { - try { - BytesRef idBytes = cmd.getIndexedId(); - if (versionsStored) { + BytesRef idBytes = cmd.getIndexedId(); + if (versionsStored) { - if (leaderLogic) { + if (leaderLogic) { - if (forwardedFromCollection && ulog.getState() == UpdateLog.State.ACTIVE) { - // forwarded from a collection but we are not buffering so strip original version and - // apply our own - // see SOLR-5308 - if (log.isInfoEnabled()) { - log.info("Removing version field from doc: {}", cmd.getId()); - } - versionOnUpdate = signedVersionOnUpdate = 0; + if (forwardedFromCollection && ulog.getState() == UpdateLog.State.ACTIVE) { + // forwarded from a collection but we are not buffering so strip original version and + // apply our own + // see SOLR-5308 + if (log.isInfoEnabled()) { + log.info("Removing version field from doc: {}", cmd.getId()); } + versionOnUpdate = signedVersionOnUpdate = 0; + } - // leaders can also be in buffering state during "migrate" API call, see SOLR-5308 - if (forwardedFromCollection - && ulog.getState() != UpdateLog.State.ACTIVE - && !isReplayOrPeersync) { - // we're not in an active state, and this update isn't from a replay, so buffer it. - if (log.isInfoEnabled()) { - log.info("Leader logic applied but update log is buffering: {}", cmd.getId()); - } - cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); - ulog.delete(cmd); - return true; + // leaders can also be in buffering state during "migrate" API call, see SOLR-5308 + if (forwardedFromCollection + && ulog.getState() != UpdateLog.State.ACTIVE + && !isReplayOrPeersync) { + // we're not in an active state, and this update isn't from a replay, so buffer it. + if (log.isInfoEnabled()) { + log.info("Leader logic applied but update log is buffering: {}", cmd.getId()); } + cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); + ulog.delete(cmd); + return true; + } - if (signedVersionOnUpdate != 0) { - Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); - long foundVersion = lastVersion == null ? -1 : lastVersion; - if ((signedVersionOnUpdate == foundVersion) - || (signedVersionOnUpdate < 0 && foundVersion < 0) - || (signedVersionOnUpdate == 1 && foundVersion > 0)) { - // we're ok if versions match, or if both are negative (all missing docs are equal), - // or if cmd specified it must exist (versionOnUpdate==1) and it does. - } else { - throw new SolrException( - ErrorCode.CONFLICT, - "version conflict for " - + cmd.getId() - + " expected=" - + signedVersionOnUpdate - + " actual=" - + foundVersion); - } + if (signedVersionOnUpdate != 0) { + Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); + long foundVersion = lastVersion == null ? -1 : lastVersion; + if ((signedVersionOnUpdate == foundVersion) + || (signedVersionOnUpdate < 0 && foundVersion < 0) + || (signedVersionOnUpdate == 1 && foundVersion > 0)) { + // we're ok if versions match, or if both are negative (all missing docs are equal), + // or if cmd specified it must exist (versionOnUpdate==1) and it does. + } else { + throw new SolrException( + ErrorCode.CONFLICT, + "version conflict for " + + cmd.getId() + + " expected=" + + signedVersionOnUpdate + + " actual=" + + foundVersion); } + } - long version = vinfo.getNewClock(); - cmd.setVersion(-version); - } else { - cmd.setVersion(-versionOnUpdate); + long version = vinfo.getNewClock(); + cmd.setVersion(-version); + } else { + cmd.setVersion(-versionOnUpdate); - if (ulog.getState() != UpdateLog.State.ACTIVE && isReplayOrPeersync == false) { - // we're not in an active state, and this update isn't from a replay, so buffer it. - cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); - ulog.delete(cmd); - return true; - } + if (ulog.getState() != UpdateLog.State.ACTIVE && isReplayOrPeersync == false) { + // we're not in an active state, and this update isn't from a replay, so buffer it. + cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING); + ulog.delete(cmd); + return true; + } - // if we aren't the leader, then we need to check that updates were not re-ordered - // we need to check the specific version for this id. - Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); - if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) { - // This update is a repeat, or was reordered. We need to drop this update. - if (log.isDebugEnabled()) { - log.debug("Dropping delete update due to version {}", idBytes.utf8ToString()); - } - return true; + // if we aren't the leader, then we need to check that updates were not re-ordered + // we need to check the specific version for this id. + Long lastVersion = vinfo.lookupVersion(cmd.getIndexedId()); + if (lastVersion != null && Math.abs(lastVersion) >= versionOnUpdate) { + // This update is a repeat, or was reordered. We need to drop this update. + if (log.isDebugEnabled()) { + log.debug("Dropping delete update due to version {}", idBytes.utf8ToString()); } + return true; + } - if (!isSubShardLeader - && replicaType == Replica.Type.TLOG - && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { - cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); - } + if (!isSubShardLeader + && replicaType == Replica.Type.TLOG + && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { + cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); } } - - doLocalDelete(cmd); - return false; - } finally { - bucket.unlock(); } + + doLocalDelete(cmd); + return false; } private static boolean leaderLogicWithVersionIntegrityCheck( @@ -1202,7 +1165,7 @@ protected void doLocalCommit(CommitUpdateCommand cmd) throws IOException { if (vinfo != null) { long commitVersion = vinfo.getNewClock(); cmd.setVersion(commitVersion); - vinfo.lockForUpdate(); + getUpdateLocks().lockForUpdate(); } try { @@ -1221,7 +1184,7 @@ protected void doLocalCommit(CommitUpdateCommand cmd) throws IOException { } finally { if (vinfo != null) { - vinfo.unlockForUpdate(); + getUpdateLocks().unlockForUpdate(); } } } diff --git a/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java b/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java index 65d2491ed1e..3e3abb4e312 100644 --- a/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java +++ b/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java @@ -25,9 +25,9 @@ import java.util.concurrent.Semaphore; import org.apache.solr.common.util.ExecutorUtil; -public class OrderedExecutor implements Executor { +public class OrderedExecutor implements Executor { private final ExecutorService delegate; - private final SparseStripedLock sparseStripedLock; + private final SparseStripedLock sparseStripedLock; public OrderedExecutor(int numThreads, ExecutorService delegate) { this.delegate = delegate; @@ -51,7 +51,7 @@ public void execute(Runnable runnable) { * @param command the runnable task * @throws RejectedExecutionException if this task cannot be accepted for execution */ - public void execute(Integer lockId, Runnable command) { + public void execute(T lockId, Runnable command) { try { sparseStripedLock.add(lockId); } catch (InterruptedException e) { diff --git a/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml index 0372460c2b2..115fea2368b 100644 --- a/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml +++ b/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml @@ -30,7 +30,6 @@ ${solr.ulog.dir:} - ${solr.ulog.numVersionBuckets:65536} diff --git a/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java index dc430437121..0014260001a 100644 --- a/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java +++ b/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java @@ -18,8 +18,6 @@ package org.apache.solr.update.processor; import static org.hamcrest.CoreMatchers.is; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doReturn; import java.io.IOException; import java.util.ArrayList; @@ -39,25 +37,18 @@ import org.apache.solr.update.AddUpdateCommand; import org.apache.solr.update.DeleteUpdateCommand; import org.apache.solr.update.SolrCmdDistributor; -import org.apache.solr.update.TimedVersionBucket; +import org.apache.solr.update.UpdateLocks; import org.apache.solr.update.UpdateLog; -import org.apache.solr.update.VersionInfo; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; import org.junit.Test; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnit; -import org.mockito.junit.MockitoRule; public class DistributedUpdateProcessorTest extends SolrTestCaseJ4 { - @Rule public MockitoRule rule = MockitoJUnit.rule(); private static ExecutorService executor; @BeforeClass public static void beforeClass() throws Exception { - assumeWorkingMockito(); executor = ExecutorUtil.newMDCAwareCachedThreadPool(getClassName()); System.setProperty("enable.update.log", "true"); initCore( @@ -66,10 +57,8 @@ public static void beforeClass() throws Exception { } @AfterClass - public static void AfterClass() { - if (null != executor) { // may not have been initialized due to lack of mockito - executor.shutdown(); - } + public static void afterClass() { + ExecutorUtil.shutdownAndAwaitTermination(executor); System.clearProperty("enable.update.log"); } @@ -164,36 +153,12 @@ public void testStatusCodeOnDistribError_NotSolrException() { */ private int runCommands( int threads, - int versionBucketLockTimeoutMs, + int docLockTimeoutMs, SolrQueryRequest req, Function function) throws IOException { - try (DistributedUpdateProcessor processor = - new DistributedUpdateProcessor(req, null, null, null)) { - if (versionBucketLockTimeoutMs > 0) { - // use TimedVersionBucket with versionBucketLockTimeoutMs - VersionInfo vinfo = Mockito.spy(processor.vinfo); - processor.vinfo = vinfo; - - doReturn( - new TimedVersionBucket() { - /** simulate the case: it takes 5 seconds to add the doc */ - @Override - protected boolean tryLock(int lockTimeoutMs) { - boolean locked = super.tryLock(versionBucketLockTimeoutMs); - if (locked) { - try { - Thread.sleep(5000); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - return locked; - } - }) - .when(vinfo) - .bucket(anyInt()); - } + + try (DistributedUpdateProcessor processor = newDurp(req, docLockTimeoutMs)) { CountDownLatch latch = new CountDownLatch(1); Collection> futures = new ArrayList<>(); for (int t = 0; t < threads; ++t) { @@ -220,4 +185,45 @@ protected boolean tryLock(int lockTimeoutMs) { return succeeded; } } + + private static DistributedUpdateProcessor newDurp(SolrQueryRequest req, long lockTimeoutMs) { + if (lockTimeoutMs <= 0) { + // default + return new DistributedUpdateProcessor(req, null, null); + } + // customize UpdateLocks with the provided timeout. And simulate docs taking longer to index + final var sleepMs = lockTimeoutMs + 1000; + assert sleepMs > lockTimeoutMs; + var sleepUrp = + new UpdateRequestProcessor(null) { + @Override + public void processAdd(AddUpdateCommand cmd) throws IOException { + sleep(); + super.processAdd(cmd); + } + + @Override + public void processDelete(DeleteUpdateCommand cmd) throws IOException { + sleep(); + super.processDelete(cmd); + } + + private void sleep() { + try { + Thread.sleep(sleepMs); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }; + + return new DistributedUpdateProcessor(req, null, null, sleepUrp) { + UpdateLocks updateLocks = new UpdateLocks(lockTimeoutMs); + + @Override + protected UpdateLocks getUpdateLocks() { + return updateLocks; + } + }; + } } diff --git a/solr/core/src/test/org/apache/solr/util/OrderedExecutorTest.java b/solr/core/src/test/org/apache/solr/util/OrderedExecutorTest.java index 4b49baa6e01..41099cf9c9c 100644 --- a/solr/core/src/test/org/apache/solr/util/OrderedExecutorTest.java +++ b/solr/core/src/test/org/apache/solr/util/OrderedExecutorTest.java @@ -39,8 +39,8 @@ public class OrderedExecutorTest extends SolrTestCase { @Test public void testExecutionInOrder() { - OrderedExecutor orderedExecutor = - new OrderedExecutor(10, ExecutorUtil.newMDCAwareCachedThreadPool("executeInOrderTest")); + OrderedExecutor orderedExecutor = + new OrderedExecutor<>(10, ExecutorUtil.newMDCAwareCachedThreadPool("executeInOrderTest")); IntBox intBox = new IntBox(); for (int i = 0; i < 100; i++) { orderedExecutor.execute(1, () -> intBox.value++); @@ -53,8 +53,8 @@ public void testExecutionInOrder() { public void testLockWhenQueueIsFull() { final ExecutorService controlExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("testLockWhenQueueIsFull_control"); - final OrderedExecutor orderedExecutor = - new OrderedExecutor( + final OrderedExecutor orderedExecutor = + new OrderedExecutor<>( 10, ExecutorUtil.newMDCAwareCachedThreadPool("testLockWhenQueueIsFull_test")); try { @@ -111,8 +111,8 @@ public void testRunInParallel() { final ExecutorService controlExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("testRunInParallel_control"); - final OrderedExecutor orderedExecutor = - new OrderedExecutor( + final OrderedExecutor orderedExecutor = + new OrderedExecutor<>( parallelism, ExecutorUtil.newMDCAwareCachedThreadPool("testRunInParallel_test")); try { @@ -216,8 +216,8 @@ public void testStress() { base.put(i, i); run.put(i, i); } - OrderedExecutor orderedExecutor = - new OrderedExecutor(10, ExecutorUtil.newMDCAwareCachedThreadPool("testStress")); + OrderedExecutor orderedExecutor = + new OrderedExecutor<>(10, ExecutorUtil.newMDCAwareCachedThreadPool("testStress")); for (int i = 0; i < 1000; i++) { int key = random().nextInt(N); base.put(key, base.get(key) + 1); @@ -233,8 +233,8 @@ private static class IntBox { @Test public void testMaxSize() throws InterruptedException { - OrderedExecutor orderedExecutor = - new OrderedExecutor(1, ExecutorUtil.newMDCAwareCachedThreadPool("single")); + OrderedExecutor orderedExecutor = + new OrderedExecutor<>(1, ExecutorUtil.newMDCAwareCachedThreadPool("single")); CountDownLatch isRunning = new CountDownLatch(1); CountDownLatch blockingLatch = new CountDownLatch(1); diff --git a/solr/prometheus-exporter/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/prometheus-exporter/src/test-files/solr/collection1/conf/solrconfig.xml index bbc9b68628b..6ca22087fe4 100644 --- a/solr/prometheus-exporter/src/test-files/solr/collection1/conf/solrconfig.xml +++ b/solr/prometheus-exporter/src/test-files/solr/collection1/conf/solrconfig.xml @@ -35,7 +35,6 @@ ${solr.ulog.dir:} - ${solr.ulog.numVersionBuckets:65536} diff --git a/solr/server/solr/configsets/_default/conf/solrconfig.xml b/solr/server/solr/configsets/_default/conf/solrconfig.xml index 92896c6d3f7..825993191ae 100644 --- a/solr/server/solr/configsets/_default/conf/solrconfig.xml +++ b/solr/server/solr/configsets/_default/conf/solrconfig.xml @@ -256,16 +256,9 @@ is recommended (see below). "dir" - the target directory for transaction logs, defaults to the solr data directory. - "numVersionBuckets" - sets the number of buckets used to keep - track of max version values when checking for re-ordered - updates; increase this value to reduce the cost of - synchronizing access to version buckets during high-volume - indexing, this requires 8 bytes (long) * numVersionBuckets - of heap space per Solr core. --> ${solr.ulog.dir:} - ${solr.ulog.numVersionBuckets:65536} ${solr.ulog.dir:} - ${solr.ulog.numVersionBuckets:65536}