This is an automated email from the ASF dual-hosted git repository.

epugh pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new 87dff3e7b56 SOLR-17851: Remove BlobHandler and .system collection 
support (#3464)
87dff3e7b56 is described below

commit 87dff3e7b56b7017119925036f7a4ebd77a32965
Author: Eric Pugh <[email protected]>
AuthorDate: Fri Aug 15 11:10:30 2025 -0400

    SOLR-17851: Remove BlobHandler and .system collection support (#3464)
    
    Remove code that was deprecated in 9x.
---
 dev-docs/overseer/overseer.adoc                    |   2 -
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/cli/CreateTool.java   |   5 +-
 .../src/java/org/apache/solr/cloud/Overseer.java   | 151 ---------
 .../cloud/api/collections/CreateCollectionCmd.java |  11 +-
 .../api/collections/ReindexCollectionCmd.java      |   4 +-
 .../java/org/apache/solr/handler/BlobHandler.java  | 373 ---------------------
 .../solr/handler/admin/api/CreateCollection.java   |  47 ---
 .../solr/handler/admin/api/GetBlobInfoAPI.java     |  64 ----
 .../solr/handler/admin/api/UploadBlobAPI.java      |  48 ---
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  54 ---
 solr/core/src/resources/SystemCollectionSchema.xml |  38 ---
 .../src/resources/SystemCollectionSolrConfig.xml   |  19 --
 .../solr/cloud/SystemCollectionCompatTest.java     | 258 --------------
 .../org/apache/solr/handler/TestBlobHandler.java   | 210 ------------
 .../solr/handler/TestSystemCollAutoCreate.java     |  27 --
 .../solr/handler/admin/TestApiFramework.java       |  22 --
 .../solr/handler/admin/V2BlobAPIMappingTest.java   |  57 ----
 .../handler/designer/TestSchemaDesignerAPI.java    |   2 +-
 .../pages/requesthandlers-searchcomponents.adoc    |   7 +
 .../modules/configuration-guide/pages/v2-api.adoc  |   3 +-
 .../pages/major-changes-in-solr-10.adoc            |   7 +-
 .../solr/common/params/CollectionAdminParams.java  |   3 -
 .../apache/solr/common/params/CommonParams.java    |   3 -
 24 files changed, 20 insertions(+), 1397 deletions(-)

diff --git a/dev-docs/overseer/overseer.adoc b/dev-docs/overseer/overseer.adoc
index 350096b6633..b08aed5faa6 100644
--- a/dev-docs/overseer/overseer.adoc
+++ b/dev-docs/overseer/overseer.adoc
@@ -161,8 +161,6 @@ Overseer `start()` method creates and starts as 
`OverseerThread`’s:
 * <<OverseerCollectionConfigSetProcessor>> (consuming collection queue which 
includes the identical config set queue),
 * OverseerTriggerThread (for Autoscaling, not discussed in this doc)
 
-start() also verifies that the system collection ”.system” is in good shape 
(segment/Lucene versions).
-
 [[ClusterStateUpdater,ClusterStateUpdater]]
 ==== Inner class ClusterStateUpdater
 
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0c73d59679d..3baf170ec67 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -141,6 +141,8 @@ Deprecation Removals
 
 * SOLR-17839: Removed PreAnalyzedFields (Ishan Chattopadhyaya)
 
+* SOLR-17851: Removed BlobHandler and .system collection support (Eric Pugh)
+
 Dependency Upgrades
 ---------------------
 
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 e9bf1dd4b1a..5ce1fdbfb8c 100644
--- a/solr/core/src/java/org/apache/solr/cli/CreateTool.java
+++ b/solr/core/src/java/org/apache/solr/cli/CreateTool.java
@@ -40,7 +40,6 @@ import 
org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
 import org.apache.solr.cloud.ZkConfigSetService;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.ConfigSetService;
@@ -259,9 +258,7 @@ public class CreateTool extends ToolBase {
                 .getZkClient()
                 .exists("/configs/" + confName, true);
 
-    if (CollectionAdminParams.SYSTEM_COLL.equals(collectionName)) {
-      // do nothing
-    } else if (configExistsInZk) {
+    if (configExistsInZk) {
       echo("Re-using existing configuration directory " + confName);
     } else { // if (confdir != null && !confdir.trim().isEmpty()) {
       if (confName == null || confName.trim().isEmpty()) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java 
b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 91877fe790e..b1a9f72c1f7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -23,23 +23,13 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayDeque;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiConsumer;
-import org.apache.lucene.util.Version;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
-import org.apache.solr.client.solrj.impl.Http2SolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
 import org.apache.solr.cloud.overseer.ClusterStateMutator;
@@ -55,16 +45,12 @@ import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.Compressor;
 import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.StrUtils;
@@ -788,148 +774,11 @@ public class Overseer implements SolrCloseable {
     updaterThread.start();
     ccThread.start();
 
-    systemCollectionCompatCheck(
-        new BiConsumer<>() {
-          boolean firstPair = true;
-
-          @Override
-          public void accept(String s, Object o) {
-            if (firstPair) {
-              log.warn(
-                  "WARNING: Collection '.system' may need re-indexing due to 
compatibility issues listed below. See REINDEXCOLLECTION documentation for more 
details.");
-              firstPair = false;
-            }
-            log.warn("WARNING: *\t{}:\t{}", s, o);
-          }
-        });
-
     getCoreContainer().getClusterSingletons().startClusterSingletons();
 
     assert ObjectReleaseTracker.track(this);
   }
 
-  public void systemCollectionCompatCheck(final BiConsumer<String, Object> 
consumer) {
-    ClusterState clusterState = zkController.getClusterState();
-    if (clusterState == null) {
-      log.warn("Unable to check back-compat of .system collection - can't 
obtain ClusterState.");
-      return;
-    }
-    DocCollection coll = 
clusterState.getCollectionOrNull(CollectionAdminParams.SYSTEM_COLL);
-    if (coll == null) {
-      return;
-    }
-    // check that all shard leaders are active
-    boolean allActive = true;
-    for (Slice s : coll.getActiveSlices()) {
-      if (s.getLeader() == null || 
!s.getLeader().isActive(clusterState.getLiveNodes())) {
-        allActive = false;
-        break;
-      }
-    }
-    if (allActive) {
-      doCompatCheck(consumer);
-    } else {
-      // wait for all leaders to become active and then check
-      zkController.zkStateReader.registerCollectionStateWatcher(
-          CollectionAdminParams.SYSTEM_COLL,
-          (liveNodes, state) -> {
-            boolean active = true;
-            if (state == null || liveNodes.isEmpty()) {
-              return true;
-            }
-            for (Slice s : state.getActiveSlices()) {
-              if (s.getLeader() == null || !s.getLeader().isActive(liveNodes)) 
{
-                active = false;
-                break;
-              }
-            }
-            if (active) {
-              doCompatCheck(consumer);
-            }
-            return active;
-          });
-    }
-  }
-
-  private void doCompatCheck(BiConsumer<String, Object> consumer) {
-    if (systemCollCompatCheck) {
-      systemCollCompatCheck = false;
-    } else {
-      return;
-    }
-
-    try (var solrClient =
-            new Http2SolrClient.Builder()
-                .withHttpClient(getCoreContainer().getDefaultHttpSolrClient())
-                .withIdleTimeout(30000, TimeUnit.MILLISECONDS)
-                .build();
-        var client =
-            new CloudHttp2SolrClient.Builder(
-                    
Collections.singletonList(getZkController().getZkServerAddress()),
-                    Optional.empty())
-                .withHttpClient(solrClient)
-                .build()) {
-      CollectionAdminRequest.ColStatus req =
-          
CollectionAdminRequest.collectionStatus(CollectionAdminParams.SYSTEM_COLL)
-              .setWithSegments(true)
-              .setWithFieldInfo(true);
-      CollectionAdminResponse rsp = req.process(client);
-      NamedList<?> status = (NamedList<?>) 
rsp.getResponse().get(CollectionAdminParams.SYSTEM_COLL);
-      Collection<?> nonCompliant = (Collection<?>) 
status.get("schemaNonCompliant");
-      if (!nonCompliant.contains("(NONE)")) {
-        consumer.accept("indexFieldsNotMatchingSchema", nonCompliant);
-      }
-      Set<Integer> segmentCreatedMajorVersions = new HashSet<>();
-      Set<String> segmentVersions = new HashSet<>();
-      int currentMajorVersion = Version.LATEST.major;
-      String currentVersion = Version.LATEST.toString();
-      segmentVersions.add(currentVersion);
-      segmentCreatedMajorVersions.add(currentMajorVersion);
-      NamedList<?> shards = (NamedList<?>) status.get("shards");
-      for (Map.Entry<String, ?> entry : shards) {
-        NamedList<?> leader = (NamedList<?>) ((NamedList<?>) 
entry.getValue()).get("leader");
-        if (leader == null) {
-          continue;
-        }
-        NamedList<?> segInfos = (NamedList<?>) leader.get("segInfos");
-        if (segInfos == null) {
-          continue;
-        }
-        NamedList<?> infos = (NamedList<?>) segInfos.get("info");
-        if (((Number) infos.get("numSegments")).intValue() > 0) {
-          segmentVersions.add(infos.get("minSegmentLuceneVersion").toString());
-        }
-        if (infos.get("commitLuceneVersion") != null) {
-          segmentVersions.add(infos.get("commitLuceneVersion").toString());
-        }
-        NamedList<?> segmentInfos = (NamedList<?>) segInfos.get("segments");
-        segmentInfos.forEach(
-            (k, v) -> {
-              NamedList<?> segment = (NamedList<?>) v;
-              segmentVersions.add(segment.get("version").toString());
-              if (segment.get("minVersion") != null) {
-                segmentVersions.add(segment.get("version").toString());
-              }
-              if (segment.get("createdVersionMajor") != null) {
-                segmentCreatedMajorVersions.add(
-                    ((Number) segment.get("createdVersionMajor")).intValue());
-              }
-            });
-      }
-      if (segmentVersions.size() > 1) {
-        consumer.accept("differentSegmentVersions", segmentVersions);
-        consumer.accept("currentLuceneVersion", currentVersion);
-      }
-      if (segmentCreatedMajorVersions.size() > 1) {
-        consumer.accept("differentMajorSegmentVersions", 
segmentCreatedMajorVersions);
-        consumer.accept("currentLuceneMajorVersion", currentMajorVersion);
-      }
-
-    } catch (SolrServerException | IOException e) {
-      log.warn("Unable to perform back-compat check of .system collection", e);
-    }
-  }
-
   /** Start {@link ClusterSingleton} plugins when we become the leader. */
 
   /** Stop {@link ClusterSingleton} plugins when we lose leadership. */
diff --git 
a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
 
b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 2b8a5007bf1..e216a7726bc 100644
--- 
a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ 
b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -74,7 +74,6 @@ import org.apache.solr.common.cloud.ReplicaPosition;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
-import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -618,13 +617,9 @@ public class CreateCollectionCmd implements 
CollApiCmds.CollectionApiCommand {
       List<String> configNames = null;
       configNames = ccc.getCoreContainer().getConfigSetService().listConfigs();
       if (configNames.contains(DEFAULT_CONFIGSET_NAME)) {
-        if (CollectionAdminParams.SYSTEM_COLL.equals(coll)) {
-          return coll;
-        } else {
-          String intendedConfigSetName = 
getSuffixedNameForAutoGeneratedConfigSet(coll);
-          copyDefaultConfigSetTo(configNames, intendedConfigSetName);
-          return intendedConfigSetName;
-        }
+        String intendedConfigSetName = 
getSuffixedNameForAutoGeneratedConfigSet(coll);
+        copyDefaultConfigSetTo(configNames, intendedConfigSetName);
+        return intendedConfigSetName;
       } else if (configNames != null && configNames.size() == 1) {
         configName = configNames.get(0);
         // no config set named, but there is only 1 - use it
diff --git 
a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
 
b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index 217ce0228c7..a36f28ebf71 100644
--- 
a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ 
b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -433,9 +433,7 @@ public class ReindexCollectionCmd implements 
CollApiCmds.CollectionApiCommand {
               + "id=\"topic_"
               + targetCollection
               + "\","
-              +
-              // some of the documents eg. in .system contain large blobs
-              "rows=\""
+              + "rows=\""
               + batchSize
               + "\","
               + "initialCheckpoint=\"0\"))))");
diff --git a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java 
b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
deleted file mode 100644
index fff7a63d55d..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
+++ /dev/null
@@ -1,373 +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.handler;
-
-import static java.util.Collections.singletonMap;
-import static org.apache.solr.common.params.CommonParams.ID;
-import static org.apache.solr.common.params.CommonParams.JSON;
-import static org.apache.solr.common.params.CommonParams.SORT;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.invoke.MethodHandles;
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.io.input.BoundedInputStream;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopFieldDocs;
-import org.apache.solr.api.AnnotatedApi;
-import org.apache.solr.api.Api;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.handler.admin.api.GetBlobInfoAPI;
-import org.apache.solr.handler.admin.api.ReplicationAPIBase;
-import org.apache.solr.handler.admin.api.UploadBlobAPI;
-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.schema.FieldType;
-import org.apache.solr.search.QParser;
-import org.apache.solr.security.AuthorizationContext;
-import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.CommitUpdateCommand;
-import org.apache.solr.update.processor.UpdateRequestProcessor;
-import org.apache.solr.update.processor.UpdateRequestProcessorChain;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BlobHandler extends RequestHandlerBase
-    implements PluginInfoInitialized, PermissionNameProvider {
-  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private static final long DEFAULT_MAX_SIZE = 5 * 1024 * 1024; // 5MB
-  private long maxSize = DEFAULT_MAX_SIZE;
-
-  @Override
-  public void handleRequestBody(final SolrQueryRequest req, SolrQueryResponse 
rsp)
-      throws Exception {
-    String httpMethod = req.getHttpMethod();
-    String path = (String) req.getContext().get("path");
-    RequestHandlerUtils.setWt(req, JSON);
-
-    List<String> pieces = StrUtils.splitSmart(path, '/');
-    String blobName = null;
-    if (pieces.size() >= 3) blobName = pieces.get(2);
-
-    if ("POST".equals(httpMethod)) {
-      if (blobName == null || blobName.isEmpty()) {
-        rsp.add("error", "Name not found");
-        return;
-      }
-      String err = SolrConfigHandler.validateName(blobName);
-      if (err != null) {
-        log.warn("no blob name");
-        rsp.add("error", err);
-        return;
-      }
-      if (req.getContentStreams() == null) {
-        log.warn("no content stream");
-        rsp.add("error", "No stream");
-        return;
-      }
-
-      for (ContentStream stream : req.getContentStreams()) {
-        ByteBuffer payload;
-        try (InputStream is = boundedInputStream(stream.getStream(), maxSize)) 
{
-          payload = Utils.toByteArray(is);
-        }
-        MessageDigest m = MessageDigest.getInstance("MD5");
-        m.update(payload.array(), payload.arrayOffset() + payload.position(), 
payload.limit());
-        String md5 = new String(Hex.encodeHex(m.digest()));
-
-        int duplicateCount = req.getSearcher().count(new TermQuery(new 
Term("md5", md5)));
-        if (duplicateCount > 0) {
-          rsp.add("error", "duplicate entry");
-          forward(
-              req,
-              null,
-              new MapSolrParams(
-                  Map.of("q", "md5:" + md5, "fl", 
"id,size,version,timestamp,blobName")),
-              rsp);
-          log.warn("duplicate entry for blob : {}", blobName);
-          return;
-        }
-
-        TopFieldDocs docs =
-            req.getSearcher()
-                .search(
-                    new TermQuery(new Term("blobName", blobName)),
-                    1,
-                    new Sort(new SortField("version", SortField.Type.LONG, 
true)));
-
-        long version = 0;
-        if (docs.totalHits.value() > 0) {
-          Document doc = 
req.getSearcher().getDocFetcher().doc(docs.scoreDocs[0].doc);
-          Number n = doc.getField("version").numericValue();
-          version = n.longValue();
-        }
-        version++;
-        String id = blobName + "/" + version;
-        Map<String, Object> doc =
-            Map.of(
-                ID,
-                id,
-                CommonParams.TYPE,
-                "blob",
-                "md5",
-                md5,
-                "blobName",
-                blobName,
-                "version",
-                version,
-                "timestamp",
-                new Date(),
-                "size",
-                payload.limit(),
-                "blob",
-                payload);
-        verifyWithRealtimeGet(blobName, version, req, doc);
-        if (log.isInfoEnabled()) {
-          log.info(
-              StrUtils.formatString(
-                  "inserting new blob {0} ,size {1}, md5 {2}",
-                  doc.get(ID), String.valueOf(payload.limit()), md5));
-        }
-        indexMap(req, rsp, doc);
-        if (log.isInfoEnabled()) {
-          log.info(
-              " Successfully Added and committed a blob with id {} and size {} 
",
-              id,
-              payload.limit());
-        }
-
-        break;
-      }
-
-    } else {
-      int version = -1;
-      if (pieces.size() > 3) {
-        try {
-          version = Integer.parseInt(pieces.get(3));
-        } catch (NumberFormatException e) {
-          rsp.add("error", "Invalid version" + pieces.get(3));
-          return;
-        }
-      }
-      if 
(ReplicationAPIBase.FILE_STREAM.equals(req.getParams().get(CommonParams.WT))) {
-        if (blobName == null) {
-          throw new SolrException(
-              SolrException.ErrorCode.NOT_FOUND,
-              "Please send the request in the format 
/blob/<blobName>/<version>");
-        } else {
-          String q = "blobName:{0}";
-          if (version != -1) q = "id:{0}/{1}";
-          QParser qparser = QParser.getParser(StrUtils.formatString(q, 
blobName, version), req);
-          final TopDocs docs =
-              req.getSearcher()
-                  .search(
-                      qparser.parse(),
-                      1,
-                      new Sort(new SortField("version", SortField.Type.LONG, 
true)));
-          if (docs.totalHits.value() > 0) {
-            rsp.add(
-                ReplicationAPIBase.FILE_STREAM,
-                new SolrCore.RawWriter() {
-
-                  @Override
-                  public void write(OutputStream os) throws IOException {
-                    Document doc = 
req.getSearcher().getDocFetcher().doc(docs.scoreDocs[0].doc);
-                    IndexableField sf = doc.getField("blob");
-                    FieldType fieldType = 
req.getSchema().getField("blob").getType();
-                    ByteBuffer buf = (ByteBuffer) fieldType.toObject(sf);
-                    if (buf == null) {
-                      // should never happen unless a user wrote this document 
directly
-                      throw new SolrException(
-                          SolrException.ErrorCode.NOT_FOUND,
-                          "Invalid document . No field called blob");
-                    } else {
-                      os.write(buf.array(), buf.arrayOffset(), buf.limit());
-                    }
-                  }
-                });
-
-          } else {
-            throw new SolrException(
-                SolrException.ErrorCode.NOT_FOUND,
-                StrUtils.formatString(
-                    "Invalid combination of blobName {0} and version {1}", 
blobName, version));
-          }
-        }
-      } else {
-        String q = "*:*";
-        if (blobName != null) {
-          q = "blobName:{0}";
-          if (version != -1) {
-            q = "id:{0}/{1}";
-          }
-        }
-
-        forward(
-            req,
-            null,
-            new MapSolrParams(
-                Map.of(
-                    "q",
-                    StrUtils.formatString(q, blobName, version),
-                    "fl",
-                    "id,size,version,timestamp,blobName,md5",
-                    SORT,
-                    "version desc")),
-            rsp);
-      }
-    }
-  }
-
-  private static InputStream boundedInputStream(final InputStream is, final 
long maxLength)
-      throws IOException {
-    return new BoundedInputStream(is, maxLength) {
-      @Override
-      protected void onMaxLength(long maxLength, long count) {
-        throw new BufferOverflowException();
-      }
-    };
-  }
-
-  private void verifyWithRealtimeGet(
-      String blobName, long version, SolrQueryRequest req, Map<String, Object> 
doc) {
-    for (; ; ) {
-      SolrQueryResponse response = new SolrQueryResponse();
-      String id = blobName + "/" + version;
-      forward(req, "/get", new MapSolrParams(singletonMap(ID, id)), response);
-      if (response.getValues().get("doc") == null) {
-        // ensure that the version does not exist
-        return;
-      } else {
-        log.info("id {} already exists trying next ", id);
-        version++;
-        doc.put("version", version);
-        id = blobName + "/" + version;
-        doc.put(ID, id);
-      }
-    }
-  }
-
-  public static void indexMap(SolrQueryRequest req, SolrQueryResponse rsp, 
Map<String, Object> doc)
-      throws IOException {
-    SolrInputDocument solrDoc = new SolrInputDocument();
-    for (Map.Entry<String, Object> e : doc.entrySet()) 
solrDoc.addField(e.getKey(), e.getValue());
-    UpdateRequestProcessorChain processorChain =
-        req.getCore().getUpdateProcessorChain(req.getParams());
-    try (UpdateRequestProcessor processor = 
processorChain.createProcessor(req, rsp)) {
-      AddUpdateCommand cmd = new AddUpdateCommand(req);
-      cmd.solrDoc = solrDoc;
-      log.info("Adding doc: {}", doc);
-      processor.processAdd(cmd);
-      log.info("committing doc: {}", doc);
-      processor.processCommit(new CommitUpdateCommand(req, false));
-      processor.finish();
-    }
-  }
-
-  @Override
-  public SolrRequestHandler getSubHandler(String subPath) {
-    if (StrUtils.splitSmart(subPath, '/').size() > 4) return null;
-    return this;
-  }
-
-  //////////////////////// SolrInfoMBeans methods //////////////////////
-
-  @Override
-  public String getDescription() {
-    return "Load Jars into a system index";
-  }
-
-  @Override
-  public void init(PluginInfo info) {
-    super.init(info.initArgs);
-    if (info.initArgs != null) {
-      NamedList<?> invariants = (NamedList<?>) 
info.initArgs.get(PluginInfo.INVARIANTS);
-      if (invariants != null) {
-        Object o = invariants.get("maxSize");
-        if (o != null) {
-          maxSize = Long.parseLong(String.valueOf(o));
-          maxSize = maxSize * 1024 * 1024;
-        }
-      }
-    }
-  }
-
-  // This does not work for the general case of forwarding requests.  It 
probably currently
-  // works OK for real-time get (which is all that BlobHandler uses it for).
-  private static void forward(
-      SolrQueryRequest req, String handler, SolrParams params, 
SolrQueryResponse rsp) {
-    SolrQueryRequest r = req.subRequest(params);
-    SolrRequestInfo.getRequestInfo().addCloseHook(r); // Close as late as 
possible...
-    req.getCore().getRequestHandler(handler).handleRequest(r, rsp);
-  }
-
-  @Override
-  public Boolean registerV2() {
-    return Boolean.TRUE;
-  }
-
-  @Override
-  public Collection<Api> getApis() {
-    final List<Api> apis = new ArrayList<>();
-    apis.addAll(AnnotatedApi.getApis(new GetBlobInfoAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new UploadBlobAPI(this)));
-
-    return apis;
-  }
-
-  @Override
-  public Name getPermissionName(AuthorizationContext ctx) {
-    switch (ctx.getHttpMethod()) {
-      case "GET":
-        return Name.READ_PERM;
-      case "POST":
-        return Name.UPDATE_PERM;
-      default:
-        return null;
-    }
-  }
-}
diff --git 
a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java 
b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
index 4042d303be1..2a37aa99df6 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
@@ -43,7 +43,6 @@ import static 
org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PER
 
 import jakarta.inject.Inject;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -62,8 +61,6 @@ import 
org.apache.solr.client.solrj.request.beans.V2ApiConstants;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterProperties;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkMaintenanceUtils;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -77,8 +74,6 @@ import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.jersey.PermissionName;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * V2 API for creating a SolrCLoud collection
@@ -109,13 +104,6 @@ public class CreateCollection extends AdminAPIBase 
implements CreateCollectionAp
     final CoreContainer coreContainer = 
fetchAndValidateZooKeeperAwareCoreContainer();
     recordCollectionForLogAndTracing(requestBody.name, solrQueryRequest);
 
-    // We must always create a .system collection with only a single shard
-    if (CollectionAdminParams.SYSTEM_COLL.equals(requestBody.name)) {
-      requestBody.numShards = 1;
-      requestBody.shardNames = null;
-      createSysConfigSet(coreContainer);
-    }
-
     validateRequestBody(requestBody);
 
     // Populate any 'null' creation parameters that support COLLECTIONPROP 
defaults.
@@ -255,41 +243,6 @@ public class CreateCollection extends AdminAPIBase 
implements CreateCollectionAp
     return Integer.valueOf(String.valueOf(defaultValue));
   }
 
-  private static void createSysConfigSet(CoreContainer coreContainer)
-      throws KeeperException, InterruptedException {
-    SolrZkClient zk = 
coreContainer.getZkController().getZkStateReader().getZkClient();
-    ZkMaintenanceUtils.ensureExists(ZkStateReader.CONFIGS_ZKNODE, zk);
-    ZkMaintenanceUtils.ensureExists(
-        ZkStateReader.CONFIGS_ZKNODE + "/" + 
CollectionAdminParams.SYSTEM_COLL, zk);
-
-    try {
-      String path =
-          ZkStateReader.CONFIGS_ZKNODE + "/" + 
CollectionAdminParams.SYSTEM_COLL + "/schema.xml";
-      byte[] data;
-      try (InputStream inputStream =
-          
CollectionsHandler.class.getResourceAsStream("/SystemCollectionSchema.xml")) {
-        assert inputStream != null;
-        data = inputStream.readAllBytes();
-      }
-      assert data != null && data.length > 0;
-      ZkMaintenanceUtils.ensureExists(path, data, CreateMode.PERSISTENT, zk);
-      path =
-          ZkStateReader.CONFIGS_ZKNODE
-              + "/"
-              + CollectionAdminParams.SYSTEM_COLL
-              + "/solrconfig.xml";
-      try (InputStream inputStream =
-          
CollectionsHandler.class.getResourceAsStream("/SystemCollectionSolrConfig.xml"))
 {
-        assert inputStream != null;
-        data = inputStream.readAllBytes();
-      }
-      assert data != null && data.length > 0;
-      ZkMaintenanceUtils.ensureExists(path, data, CreateMode.PERSISTENT, zk);
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-    }
-  }
-
   public static CreateCollectionRequestBody createRequestBodyFromV1Params(
       SolrParams params, boolean nameRequired) {
     final var requestBody = new CreateCollectionRequestBody();
diff --git 
a/solr/core/src/java/org/apache/solr/handler/admin/api/GetBlobInfoAPI.java 
b/solr/core/src/java/org/apache/solr/handler/admin/api/GetBlobInfoAPI.java
deleted file mode 100644
index 29345c12b88..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/GetBlobInfoAPI.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.handler.admin.api;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
-import static org.apache.solr.security.PermissionNameProvider.Name.READ_PERM;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.BlobHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-
-/**
- * V2 APIs for fetching blob(s) and their metadata
- *
- * <p>These APIs (GET /v2/collections/.system/blob/*) is analogous to the v1 
GET
- * /solr/.system/blob/* APIs.
- */
-public class GetBlobInfoAPI {
-  private BlobHandler blobHandler;
-
-  public GetBlobInfoAPI(BlobHandler blobHandler) {
-    this.blobHandler = blobHandler;
-  }
-
-  @EndPoint(
-      path = {"/blob"},
-      method = GET,
-      permission = READ_PERM)
-  public void getAllBlobs(SolrQueryRequest req, SolrQueryResponse rsp) {
-    blobHandler.handleRequest(req, rsp);
-  }
-
-  @EndPoint(
-      path = {"/blob/{blobName}"},
-      method = GET,
-      permission = READ_PERM)
-  public void getBlobByName(SolrQueryRequest req, SolrQueryResponse rsp) {
-    blobHandler.handleRequest(req, rsp);
-  }
-
-  @EndPoint(
-      path = {"/blob/{blobName}/{blobVersion}"},
-      method = GET,
-      permission = READ_PERM)
-  public void getVersionedBlobByName(SolrQueryRequest req, SolrQueryResponse 
rsp) {
-    blobHandler.handleRequest(req, rsp);
-  }
-}
diff --git 
a/solr/core/src/java/org/apache/solr/handler/admin/api/UploadBlobAPI.java 
b/solr/core/src/java/org/apache/solr/handler/admin/api/UploadBlobAPI.java
deleted file mode 100644
index 2488a109648..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/UploadBlobAPI.java
+++ /dev/null
@@ -1,48 +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.handler.admin.api;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.security.PermissionNameProvider.Name.UPDATE_PERM;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.BlobHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-
-/**
- * V2 API for uploading blobs into Solr's .system blobstore
- *
- * <p>This API (POST /v2/collections/.system/blob/blobName) is analogous to 
the v1 POST
- * /solr/.system/blob/blobName API.
- */
-public class UploadBlobAPI {
-  private final BlobHandler blobHandler;
-
-  public UploadBlobAPI(BlobHandler blobHandler) {
-    this.blobHandler = blobHandler;
-  }
-
-  @EndPoint(
-      path = {"/blob/{blobName}"},
-      method = POST,
-      permission = UPDATE_PERM)
-  public void uploadBlob(SolrQueryRequest req, SolrQueryResponse rsp) {
-    blobHandler.handleRequest(req, rsp);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java 
b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 9a45fb2682e..7be425273a6 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -19,11 +19,6 @@ package org.apache.solr.servlet;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.common.params.CollectionAdminParams.SYSTEM_COLL;
-import static 
org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.common.params.CoreAdminParams.ACTION;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.FORWARD;
 import static org.apache.solr.servlet.SolrDispatchFilter.Action.PASSTHROUGH;
@@ -56,8 +51,6 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.function.Supplier;
 import net.jcip.annotations.ThreadSafe;
 import org.apache.http.Header;
@@ -107,7 +100,6 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ContentStreamHandlerBase;
-import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.request.SolrRequestHandler;
@@ -301,8 +293,6 @@ public class HttpSolrCall {
               return;
             }
           }
-          // core is not available locally or remotely
-          autoCreateSystemColl(collectionName);
           if (action != null) return;
         }
       }
@@ -376,50 +366,6 @@ public class HttpSolrCall {
     return logic.get();
   }
 
-  protected void autoCreateSystemColl(String corename) throws Exception {
-    if (core == null
-        && SYSTEM_COLL.equals(corename)
-        && "POST".equals(req.getMethod())
-        && 
!cores.getZkController().getClusterState().hasCollection(SYSTEM_COLL)) {
-      log.info("Going to auto-create {} collection", SYSTEM_COLL);
-      SolrQueryResponse rsp = new SolrQueryResponse();
-      String repFactor =
-          String.valueOf(
-              Math.min(3, 
cores.getZkController().getClusterState().getLiveNodes().size()));
-      cores
-          .getCollectionsHandler()
-          .handleRequestBody(
-              new LocalSolrQueryRequest(
-                  null,
-                  new ModifiableSolrParams()
-                      .add(ACTION, CREATE.toString())
-                      .add(NAME, SYSTEM_COLL)
-                      .add(REPLICATION_FACTOR, repFactor)),
-              rsp);
-      if (rsp.getValues().get("success") == null) {
-        throw new SolrException(
-            ErrorCode.SERVER_ERROR,
-            "Could not auto-create "
-                + SYSTEM_COLL
-                + " collection: "
-                + Utils.toJSONString(rsp.getValues()));
-      }
-
-      try {
-        cores
-            .getZkController()
-            .getZkStateReader()
-            .waitForState(SYSTEM_COLL, 3, TimeUnit.SECONDS, Objects::nonNull);
-      } catch (TimeoutException e) {
-        throw new SolrException(
-            ErrorCode.SERVER_ERROR,
-            "Could not find " + SYSTEM_COLL + " collection even after 3 
seconds");
-      }
-
-      action = RETRY;
-    }
-  }
-
   /**
    * Resolves the parameter as a potential comma delimited list of 
collections, and resolves aliases
    * too. One level of aliases pointing to another alias is supported. 
De-duplicates and retains the
diff --git a/solr/core/src/resources/SystemCollectionSchema.xml 
b/solr/core/src/resources/SystemCollectionSchema.xml
deleted file mode 100644
index a21aee2d012..00000000000
--- a/solr/core/src/resources/SystemCollectionSchema.xml
+++ /dev/null
@@ -1,38 +0,0 @@
-<?xml version="1.0" ?>
-<schema name="_system collection or core" version="1.7">
-  <fieldtype name="string"  class="solr.StrField" sortMissingLast="true" 
omitNorms="true"/>
-  <fieldType name="long" class="solr.LongPointField" positionIncrementGap="0"/>
-  <fieldType name="double" class="solr.DoublePointField" 
positionIncrementGap="0"/>
-  <fieldType name="bytes" class="solr.BinaryField"/>
-  <fieldType name="date" class="solr.DatePointField"/>
-  <fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
-    <analyzer>
-      <tokenizer class="solr.WhitespaceTokenizerFactory"/>
-    </analyzer>
-  </fieldType>
-  <uniqueKey>id</uniqueKey>
-  <field name="id"   type="string"   indexed="true"  stored="true"  
multiValued="false" required="true"/>
-  <field name="_version_" type="long"     indexed="true"  stored="true"/>
-  <field name="type"   type="string"   indexed="true"  stored="true"  
multiValued="false" required="true"/>
-  <!-- blob repository fields -->
-  <field name="md5"   type="string"   indexed="true"  stored="true"  
multiValued="false"/>
-  <field name="blob"      type="bytes"   indexed="false" stored="true"  
multiValued="false" />
-  <field name="size"      type="long"   indexed="true" stored="true"  
multiValued="false" />
-  <field name="version"   type="long"     indexed="true"  stored="true"  
multiValued="false" />
-  <field name="timestamp"   type="date"   indexed="true"  stored="true"  
multiValued="false" />
-  <field name="blobName"      type="string"   indexed="true"  stored="true"  
multiValued="false" />
-
-  <!-- general purpose fields -->
-  <dynamicField name="*_s"  type="string"  indexed="true"  stored="true" />
-  <dynamicField name="*_ss"  type="string"  indexed="true"  multiValued="true" 
 stored="true" />
-  <dynamicField name="*_str"  type="string"  indexed="false" stored="true" />
-  <dynamicField name="*_strs"  type="string"  indexed="false"  
multiValued="true" stored="true" />
-  <dynamicField name="*_bin"  type="bytes"  indexed="false"  
multiValued="false" stored="true" />
-  <dynamicField name="*_t"  type="text_ws"  indexed="true"  stored="true" />
-  <dynamicField name="*_ts"  type="text_ws"  indexed="true"  
multiValued="true" stored="true" />
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
-  <dynamicField name="*_ls" type="long" multiValued="true" indexed="true" 
stored="true"/>
-  <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
-  <dynamicField name="*_ds" type="double" multiValued="true" indexed="true" 
stored="true"/>
-</schema>
diff --git a/solr/core/src/resources/SystemCollectionSolrConfig.xml 
b/solr/core/src/resources/SystemCollectionSolrConfig.xml
deleted file mode 100644
index f857561d6e4..00000000000
--- a/solr/core/src/resources/SystemCollectionSolrConfig.xml
+++ /dev/null
@@ -1,19 +0,0 @@
-<?xml version="1.0" ?>
-<config>
-  <luceneMatchVersion>LATEST</luceneMatchVersion>
-  <directoryFactory name="DirectoryFactory" 
class="${solr.directoryFactory:solr.StandardDirectoryFactory}"/>
-  <updateHandler class="solr.DirectUpdateHandler2">
-    <updateLog>
-      <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog>
-    <autoCommit>
-      <maxDocs>1</maxDocs>
-      <openSearcher>true</openSearcher>
-    </autoCommit></updateHandler>
-  <requestHandler name="/select" class="solr.SearchHandler" default="true" />
-  <requestHandler name="/blob" class="solr.BlobHandler">
-    <lst name="invariants">
-      <str name="maxSize">${blob.max.size.mb:5}</str>
-    </lst>
-  </requestHandler>
-</config>
\ No newline at end of file
diff --git 
a/solr/core/src/test/org/apache/solr/cloud/SystemCollectionCompatTest.java 
b/solr/core/src/test/org/apache/solr/cloud/SystemCollectionCompatTest.java
deleted file mode 100644
index a67094098b0..00000000000
--- a/solr/core/src/test/org/apache/solr/cloud/SystemCollectionCompatTest.java
+++ /dev/null
@@ -1,258 +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.cloud;
-
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.schema.SchemaRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.schema.SchemaResponse;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.RetryUtil;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.embedded.JettySolrRunner;
-import org.apache.solr.logging.LogWatcher;
-import org.apache.solr.logging.LogWatcherConfig;
-import org.apache.solr.util.IdUtils;
-import org.apache.solr.util.TimeOut;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** */
-public class SystemCollectionCompatTest extends SolrCloudTestCase {
-  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    System.setProperty("managed.schema.mutable", "true");
-    configureCluster(2).addConfig("conf1", 
configset("cloud-managed")).configure();
-    if (!log.isWarnEnabled()) {
-      fail("Test requires that log-level is at-least WARN, but WARN is 
disabled");
-    }
-  }
-
-  private SolrCloudManager cloudManager;
-  private CloudSolrClient solrClient;
-
-  @Before
-  public void setupSystemCollection() throws Exception {
-    ZkController zkController = 
cluster.getJettySolrRunner(0).getCoreContainer().getZkController();
-    cloudManager = zkController.getSolrCloudManager();
-    solrClient =
-        new RandomizingCloudSolrClientBuilder(
-                Collections.singletonList(zkController.getZkServerAddress()), 
Optional.empty())
-            .build();
-    CollectionAdminRequest.OverseerStatus status = new 
CollectionAdminRequest.OverseerStatus();
-    CollectionAdminResponse adminResponse = status.process(solrClient);
-    String overseerLeader = (String) adminResponse.getResponse().get("leader");
-    Set<String> nodes = new 
HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-    nodes.remove(overseerLeader);
-    // put .system replicas on other nodes that the overseer
-    CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, 
null, 1, 2)
-        .setCreateNodeSet(String.join(",", nodes))
-        .process(cluster.getSolrClient());
-    cluster.waitForActiveCollection(CollectionAdminParams.SYSTEM_COLL, 1, 2);
-    // send a dummy doc to the .system collection
-    SolrInputDocument doc =
-        new SolrInputDocument("id", IdUtils.timeRandomId(), CommonParams.TYPE, 
"dummy");
-    doc.addField("time_l", cloudManager.getTimeSource().getEpochTimeNs());
-    doc.addField("timestamp", new Date());
-    solrClient.add(CollectionAdminParams.SYSTEM_COLL, doc);
-    solrClient.commit(CollectionAdminParams.SYSTEM_COLL);
-
-    Map<String, Long> coreStartTimes = new HashMap<>();
-    DocCollection coll =
-        
cloudManager.getClusterStateProvider().getCollection(CollectionAdminParams.SYSTEM_COLL);
-    for (Replica r : coll.getReplicas()) {
-      coreStartTimes.put(r.getName(), getCoreStatus(r).startTime.getTime());
-    }
-    // trigger compat report by changing the schema
-    SchemaRequest req = new SchemaRequest();
-    SchemaResponse rsp = req.process(solrClient, 
CollectionAdminParams.SYSTEM_COLL);
-    Map<String, Object> field = getSchemaField("timestamp", rsp);
-    // make some obviously incompatible changes
-    field.put("type", "string");
-    field.put("docValues", false);
-    SchemaRequest.ReplaceField replaceFieldRequest = new 
SchemaRequest.ReplaceField(field);
-    SchemaResponse.UpdateResponse replaceFieldResponse =
-        replaceFieldRequest.process(solrClient, 
CollectionAdminParams.SYSTEM_COLL);
-    assertEquals(replaceFieldResponse.toString(), 0, 
replaceFieldResponse.getStatus());
-    CollectionAdminRequest.Reload reloadRequest =
-        
CollectionAdminRequest.reloadCollection(CollectionAdminParams.SYSTEM_COLL);
-    CollectionAdminResponse response = reloadRequest.process(solrClient);
-    assertEquals(0, response.getStatus());
-    assertTrue(response.isSuccess());
-    // wait for the reload operation of all replicas to complete
-    RetryUtil.retryUntil(
-        "Timed out waiting for core to reload",
-        30,
-        1000,
-        TimeUnit.MILLISECONDS,
-        () -> {
-          boolean allReloaded = true;
-          for (Replica r : coll.getReplicas()) {
-            long previousTime = coreStartTimes.get(r.getName());
-            try {
-              long currentTime = getCoreStatus(r).startTime.getTime();
-              allReloaded = allReloaded && (previousTime < currentTime);
-            } catch (Exception e) {
-              log.warn("Error retrieving replica status of {}", 
Utils.toJSONString(r), e);
-              allReloaded = false;
-            }
-          }
-          return allReloaded;
-        });
-    cluster.waitForActiveCollection(CollectionAdminParams.SYSTEM_COLL, 1, 2);
-  }
-
-  @After
-  public void doAfter() throws Exception {
-    log.info("doAfter: deleting all collections...");
-    cluster.deleteAllCollections();
-
-    if (null != solrClient) {
-      solrClient.close();
-      solrClient = null;
-    }
-  }
-
-  private Map<String, Object> getSchemaField(String name, SchemaResponse 
schemaResponse) {
-    List<Map<String, Object>> fields = 
schemaResponse.getSchemaRepresentation().getFields();
-    for (Map<String, Object> field : fields) {
-      if (name.equals(field.get("name"))) {
-        return field;
-      }
-    }
-    return null;
-  }
-
-  @Test
-  public void testBackCompat() throws Exception {
-    if (new CollectionAdminRequest.RequestApiDistributedProcessing()
-        .process(cluster.getSolrClient())
-        .getIsCollectionApiDistributed()) {
-      log.info("Skipping test because Collection API is distributed");
-      // TODO once we completely remove Overseer, do we need to move the back 
compat check to some
-      // other place, for example to when the .system collection is opened?
-      return;
-    }
-
-    CollectionAdminRequest.OverseerStatus status = new 
CollectionAdminRequest.OverseerStatus();
-    CloudSolrClient solrClient = cluster.getSolrClient();
-    CollectionAdminResponse adminResponse = status.process(solrClient);
-    NamedList<Object> response = adminResponse.getResponse();
-    String leader = (String) response.get("leader");
-    log.info("Overseer Status indicates that the overseer is: {}", leader);
-    JettySolrRunner overseerNode = null;
-    List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
-    for (JettySolrRunner runner : jettySolrRunners) {
-      if (runner.getNodeName().equals(leader)) {
-        overseerNode = runner;
-        break;
-      }
-    }
-    assertNotNull(overseerNode);
-    LogWatcherConfig watcherCfg = new LogWatcherConfig(true, null, "WARN", 
100);
-    LogWatcher<?> watcher = LogWatcher.newRegisteredLogWatcher(watcherCfg, 
null);
-
-    watcher.reset();
-
-    // restart Overseer to trigger the back-compat check
-    if (log.isInfoEnabled()) {
-      log.info(
-          "Stopping Overseer Node: {} ({})",
-          overseerNode.getNodeName(),
-          overseerNode.getLocalPort());
-    }
-    cluster.stopJettySolrRunner(overseerNode);
-    log.info("Waiting for new overseer election...");
-    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, 
cloudManager.getTimeSource());
-    while (!timeOut.hasTimedOut()) {
-      adminResponse = status.process(solrClient);
-      response = adminResponse.getResponse();
-      String newLeader = (String) response.get("leader");
-      if (newLeader != null && !leader.equals(newLeader)) {
-        log.info("...new overseer is: {}", newLeader);
-        break;
-      }
-      timeOut.sleep(200);
-    }
-    if (timeOut.hasTimedOut()) {
-      fail("time out waiting for new Overseer leader");
-    }
-
-    TimeOut timeOut1 = new TimeOut(60, TimeUnit.SECONDS, 
cloudManager.getTimeSource());
-    boolean foundWarning = false;
-    boolean foundSchemaWarning = false;
-
-    // TODO: replace this polling logic with a LogWatcher that uses a queue we 
can await() on...
-    log.info("Polling for log watcher to detect expected log messages...");
-    while (!timeOut1.hasTimedOut()) {
-      timeOut1.sleep(1000);
-      SolrDocumentList history = watcher.getHistory(-1, null);
-      for (SolrDocument doc : history) {
-        if (!Overseer.class.getName().equals(doc.getFieldValue("logger"))) {
-          continue;
-        }
-        if (doc.getFieldValue("message").toString().contains("re-indexing")) {
-          if (log.isInfoEnabled()) {
-            log.info("Found re-indexing message: {}", 
doc.getFieldValue("message"));
-          }
-          foundWarning = true;
-        }
-        if (doc.getFieldValue("message").toString().contains("timestamp")) {
-          if (log.isInfoEnabled()) {
-            log.info("Found timestamp message: {}", 
doc.getFieldValue("message"));
-          }
-          foundSchemaWarning = true;
-        }
-      }
-      if (foundWarning && foundSchemaWarning) {
-        break;
-      }
-    }
-    log.info(
-        "Done polling log watcher: foundWarning={} foundSchemaWarning={}",
-        foundWarning,
-        foundSchemaWarning);
-    assertTrue("re-indexing warning not found", foundWarning);
-    assertTrue("timestamp field incompatibility warning not found", 
foundSchemaWarning);
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java 
b/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
deleted file mode 100644
index c24d092c65b..00000000000
--- a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
+++ /dev/null
@@ -1,210 +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.handler;
-
-import static java.util.Arrays.asList;
-import static org.apache.solr.common.util.Utils.fromJSONString;
-
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.entity.ByteArrayEntity;
-import org.apache.http.util.EntityUtils;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudLegacySolrClient;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.RTimer;
-import org.junit.Test;
-import org.noggit.JSONParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@ThreadLeakLingering(linger = 0)
-public class TestBlobHandler extends AbstractFullDistribZkTestBase {
-  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  @Test
-  public void doBlobHandlerTest() throws Exception {
-
-    try (SolrClient client = createNewSolrClient("", 
getBaseUrl(jettys.get(0)))) {
-      CollectionAdminResponse response1;
-      CollectionAdminRequest.Create createCollectionRequest =
-          CollectionAdminRequest.createCollection(".system", 1, 2);
-      response1 = createCollectionRequest.process(client);
-      assertEquals(0, response1.getStatus());
-      assertTrue(response1.isSuccess());
-      DocCollection sysColl = 
cloudClient.getClusterState().getCollection(".system");
-      Replica replica = 
sysColl.getActiveSlicesMap().values().iterator().next().getLeader();
-
-      String baseUrl = replica.getBaseUrl();
-      String url = baseUrl + "/.system/config/requestHandler";
-      MapWriter map = TestSolrConfigHandlerConcurrent.getAsMap(url, 
cloudClient);
-      assertNotNull(map);
-      assertEquals(
-          "solr.BlobHandler", map._get(asList("config", "requestHandler", 
"/blob", "class"), null));
-      map =
-          TestSolrConfigHandlerConcurrent.getAsMap(
-              baseUrl + "/.system/schema/fields/blob", cloudClient);
-      assertNotNull(map);
-      assertEquals("blob", map._get(asList("field", "name"), null));
-      assertEquals("bytes", map._get(asList("field", "type"), null));
-
-      checkBlobPost(baseUrl, cloudClient);
-      checkBlobPostMd5(baseUrl, cloudClient);
-    }
-  }
-
-  static void checkBlobPost(String baseUrl, CloudSolrClient cloudClient) 
throws Exception {
-    String url;
-    MapWriter map;
-    byte[] bytarr = new byte[1024];
-    for (int i = 0; i < bytarr.length; i++) bytarr[i] = (byte) (i % 127);
-    byte[] bytarr2 = new byte[2048];
-    for (int i = 0; i < bytarr2.length; i++) bytarr2[i] = (byte) (i % 127);
-    String blobName = "test";
-    postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr), 1);
-    postAndCheck(cloudClient, baseUrl, blobName, ByteBuffer.wrap(bytarr2), 2);
-
-    url = baseUrl + "/.system/blob/test/1";
-    map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
-    assertEquals("" + bytarr.length, map._getStr("response/docs[0]/size"));
-
-    compareInputAndOutput(
-        baseUrl + "/.system/blob/test?wt=filestream", bytarr2, 
(CloudLegacySolrClient) cloudClient);
-    compareInputAndOutput(
-        baseUrl + "/.system/blob/test/1?wt=filestream",
-        bytarr,
-        (CloudLegacySolrClient) cloudClient);
-  }
-
-  static void checkBlobPostMd5(String baseUrl, CloudSolrClient cloudClient) 
throws Exception {
-    String blobName = "md5Test";
-    String stringValue = "MHMyugAGUxFzeqbpxVemACGbQ"; // Random string 
requires padding in md5 hash
-    String stringValueMd5 = "02d82dd5aabc47fae54ee3dd236ad83d";
-    postAndCheck(
-        cloudClient,
-        baseUrl,
-        blobName,
-        ByteBuffer.wrap(stringValue.getBytes(StandardCharsets.UTF_8)),
-        1);
-    MapWriter map =
-        TestSolrConfigHandlerConcurrent.getAsMap(
-            baseUrl + "/.system/blob/" + blobName, cloudClient);
-    assertEquals(stringValueMd5, map._getStr("response/docs[0]/md5"));
-  }
-
-  public static void createSystemCollection(SolrClient client)
-      throws SolrServerException, IOException {
-    CollectionAdminResponse response1;
-    CollectionAdminRequest.Create createCollectionRequest =
-        CollectionAdminRequest.createCollection(".system", 1, 2);
-    response1 = createCollectionRequest.process(client);
-    assertEquals(0, response1.getStatus());
-    assertTrue(response1.isSuccess());
-  }
-
-  public static void postAndCheck(
-      CloudSolrClient cloudClient, String baseUrl, String blobName, ByteBuffer 
bytes, int count)
-      throws Exception {
-    postData(cloudClient, baseUrl, blobName, bytes);
-
-    String url;
-    MapWriter map = null;
-    final RTimer timer = new RTimer();
-    int i = 0;
-    for (; i < 150; i++) { // 15 secs
-      url = baseUrl + "/.system/blob/" + blobName;
-      map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
-      String numFound = map._getStr(asList("response", "numFound"), null);
-      if (!("" + count).equals(numFound)) {
-        Thread.sleep(100);
-        continue;
-      }
-
-      assertEquals("" + bytes.limit(), map._getStr("response/docs[0]/size"));
-      return;
-    }
-    fail(
-        StrUtils.formatString(
-            "Could not successfully add blob after {0} attempts. Expecting {1} 
items. time elapsed {2}  output  for url is {3}",
-            i, count, timer.getTime(), map.toString()));
-  }
-
-  static void compareInputAndOutput(String url, byte[] bytarr, 
CloudLegacySolrClient cloudClient)
-      throws IOException {
-
-    HttpClient httpClient = cloudClient.getLbClient().getHttpClient();
-
-    HttpGet httpGet = new HttpGet(url);
-    HttpResponse entity = httpClient.execute(httpGet);
-    ByteBuffer b = Utils.toByteArray(entity.getEntity().getContent());
-    try {
-      assertEquals(b.limit(), bytarr.length);
-      for (int i = 0; i < bytarr.length; i++) {
-        assertEquals(b.get(i), bytarr[i]);
-      }
-    } finally {
-      httpGet.releaseConnection();
-    }
-  }
-
-  public static void postData(
-      CloudSolrClient cloudClient, String baseUrl, String blobName, ByteBuffer 
bytarr)
-      throws IOException {
-    HttpPost httpPost = null;
-    HttpEntity entity;
-    String response = null;
-    try {
-      httpPost = new HttpPost(baseUrl + "/.system/blob/" + blobName);
-      httpPost.setHeader("Content-Type", "application/octet-stream");
-      httpPost.setEntity(new ByteArrayEntity(bytarr.array(), 
bytarr.arrayOffset(), bytarr.limit()));
-      entity =
-          ((CloudLegacySolrClient) cloudClient)
-              .getLbClient()
-              .getHttpClient()
-              .execute(httpPost)
-              .getEntity();
-      try {
-        response = EntityUtils.toString(entity, StandardCharsets.UTF_8);
-        Map<?, ?> m = (Map<?, ?>) fromJSONString(response);
-        assertFalse("Error in posting blob " + m.toString(), 
m.containsKey("error"));
-      } catch (JSONParser.ParseException e) {
-        log.error("$ERROR$: {}", response, e);
-        fail();
-      }
-    } finally {
-      httpPost.releaseConnection();
-    }
-  }
-}
diff --git 
a/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java 
b/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
deleted file mode 100644
index 7220870aa6c..00000000000
--- a/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
+++ /dev/null
@@ -1,27 +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.handler;
-
-import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
-
-public class TestSystemCollAutoCreate extends AbstractFullDistribZkTestBase {
-  public void testAutoCreate() throws Exception {
-    TestBlobHandler.checkBlobPost(
-        cloudJettys.get(0).jetty.getBaseUrl().toExternalForm(), cloudClient);
-  }
-}
diff --git 
a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java 
b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index 2d53da3ed58..9704269bbbd 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -18,7 +18,6 @@
 package org.apache.solr.handler.admin;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.solr.api.ApiBag.EMPTY_SPEC;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static 
org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
 import static 
org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
@@ -56,7 +55,6 @@ import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.JsonSchemaValidator;
-import org.apache.solr.common.util.PathTrie;
 import org.apache.solr.common.util.ReflectMapWriter;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
@@ -284,26 +282,6 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     return rsp;
   }
 
-  public void testTrailingTemplatePaths() {
-    PathTrie<Api> registry = new PathTrie<>();
-    Api api =
-        new Api(EMPTY_SPEC) {
-          @Override
-          public void call(SolrQueryRequest req, SolrQueryResponse rsp) {}
-        };
-    Api intropsect = new ApiBag.IntrospectApi(api, false);
-    ApiBag.registerIntrospect(
-        Collections.emptyMap(), registry, "/c/.system/blob/{name}", 
intropsect);
-    ApiBag.registerIntrospect(
-        Collections.emptyMap(), registry, "/c/.system/{x}/{name}", intropsect);
-    assertEquals(
-        intropsect, 
registry.lookup("/c/.system/blob/random_string/_introspect", new HashMap<>()));
-    assertEquals(intropsect, registry.lookup("/c/.system/blob/_introspect", 
new HashMap<>()));
-    assertEquals(intropsect, registry.lookup("/c/.system/_introspect", new 
HashMap<>()));
-    assertEquals(intropsect, registry.lookup("/c/.system/v1/_introspect", new 
HashMap<>()));
-    assertEquals(intropsect, registry.lookup("/c/.system/v1/v2/_introspect", 
new HashMap<>()));
-  }
-
   private SolrQueryResponse invoke(
       PluginBag<SolrRequestHandler> reqHandlers,
       String path,
diff --git 
a/solr/core/src/test/org/apache/solr/handler/admin/V2BlobAPIMappingTest.java 
b/solr/core/src/test/org/apache/solr/handler/admin/V2BlobAPIMappingTest.java
deleted file mode 100644
index 9d1d444a012..00000000000
--- a/solr/core/src/test/org/apache/solr/handler/admin/V2BlobAPIMappingTest.java
+++ /dev/null
@@ -1,57 +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.handler.admin;
-
-import org.apache.solr.api.AnnotatedApi;
-import org.apache.solr.handler.BlobHandler;
-import org.apache.solr.handler.admin.api.GetBlobInfoAPI;
-import org.apache.solr.handler.admin.api.UploadBlobAPI;
-import org.junit.Test;
-
-public class V2BlobAPIMappingTest extends V2ApiMappingTest<BlobHandler> {
-  @Override
-  public void populateApiBag() {
-    apiBag.registerObject(new GetBlobInfoAPI(getRequestHandler()));
-    apiBag.registerObject(new UploadBlobAPI(getRequestHandler()));
-  }
-
-  @Override
-  public BlobHandler createUnderlyingRequestHandler() {
-    return createMock(BlobHandler.class);
-  }
-
-  @Override
-  public boolean isCoreSpecific() {
-    return true;
-  }
-
-  @Test
-  public void testGetBlobApiMappings() {
-    assertAnnotatedApiExistsFor("GET", "/blob");
-    assertAnnotatedApiExistsFor("GET", "/blob/someBlobName");
-    assertAnnotatedApiExistsFor("GET", "/blob/someBlobName/123");
-  }
-
-  @Test
-  public void testUploadBlobApiMapping() {
-    final AnnotatedApi uploadBlobApi = assertAnnotatedApiExistsFor("POST", 
"/blob/someBlobName");
-    assertEquals(1, uploadBlobApi.getCommands().keySet().size());
-    // Empty-string is the indicator for POST requests that don't use the 
explicit "command" syntax.
-    assertEquals("", uploadBlobApi.getCommands().keySet().iterator().next());
-  }
-}
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 29779e8732f..35b64f89daf 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
@@ -692,7 +692,7 @@ public class TestSchemaDesignerAPI extends 
SolrCloudTestCase implements SchemaDe
     SolrQuery query = new SolrQuery("*:*");
     query.setRows(0);
     QueryResponse qr = cluster.getSolrClient().query(collection, query);
-    // this proves the docs were stored in the blob store too
+    // this proves the docs were stored in the filestore too
     assertEquals(4, qr.getResults().getNumFound());
   }
 
diff --git 
a/solr/solr-ref-guide/modules/configuration-guide/pages/requesthandlers-searchcomponents.adoc
 
b/solr/solr-ref-guide/modules/configuration-guide/pages/requesthandlers-searchcomponents.adoc
index 59e4de21c93..3fb8b2f9e61 100644
--- 
a/solr/solr-ref-guide/modules/configuration-guide/pages/requesthandlers-searchcomponents.adoc
+++ 
b/solr/solr-ref-guide/modules/configuration-guide/pages/requesthandlers-searchcomponents.adoc
@@ -305,6 +305,13 @@ This should be considered as a last-resort option as the 
default list may change
 </searchComponent>
 ----
 
+=== Developing Search Components
+
+One of Solr's best characteristics is how pluggable it is.  You can easily 
develop your own Search Components.  Here are some tips:
+
+==== Sharing memory and disk intensive objects
+Objects that are shared by multiple cores on the same Solr node should use the 
{solr-javadocs}/core/org/apache/solr/core/CoreContainer.html#getObjectCache()[CoreContainer.getObjectCache]
 capabilty to prevent multiple copies of the same object from being loaded.
+
 
 == Update Request Handlers
 
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/v2-api.adoc 
b/solr/solr-ref-guide/modules/configuration-guide/pages/v2-api.adoc
index 79882093a6d..c7743122229 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/v2-api.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/v2-api.adoc
@@ -51,7 +51,6 @@ Following are some v2 API URL paths and path prefixes, along 
with some of the op
 |`/api/cores/_core-name_` |Reload, rename, delete, and unload a core.
 |`/api/node` |Perform overseer operation, rejoin leader election.
 |`/api/cluster` |Add role, remove role, set cluster property.
-|`/api/c/.system/blob` |Upload and download blobs and metadata.
 |===
 
 == Introspect
@@ -157,7 +156,7 @@ See the state of the cluster:
 ----
 $ curl http://localhost:8983/api/cluster
 
-{"responseHeader":{"status":0,"QTime":0},"collections":["gettingstarted",".system"]}
+{"responseHeader":{"status":0,"QTime":0},"collections":["gettingstarted"]}
 ----
 
 Set a cluster property:
diff --git 
a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc 
b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
index 2dc4958e8cd..21d36eb73a8 100644
--- 
a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
+++ 
b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
@@ -90,9 +90,8 @@ has been removed. Please use `-Dsolr.hiddenSysProps` or the 
envVar `SOLR_HIDDEN_
 
 * The legacy Circuit Breaker named `CircuitBreakerManager`, is removed. Please 
use individual Circuit Breaker plugins instead.
 
-* The `BlobRepository`, which was deprecated in 8x in favour of the 
`FileStore` approach is removed.
-Users should migrate to the `FileStore` implementation (per node stored file) 
and the still existing `BlobHandler` (across the cluster storage backed by 
`.system` collection).
-Please note this also removes the ability to share resource intensive objects 
across multiple cores as this feature was tied to the `BlobRepository` 
implementation.
+* BlobRepository and BlobHandler have both been removed in favour of the 
`FileStore` API implementation (per node stored file).
+To share resource intensive objects across multiple cores in components you 
should now use the `CoreContainer.getObjectCache` approach
 
 * The language specific Response Writers, which were deprecated in 9.8 in 
favour of more widely used formats like JSON have been removed.
 The removed writer types (invoked as part of the `wt` parameter) include 
`python`, `ruby`, `php`, and `phps`.
@@ -118,6 +117,8 @@ Nowadays, the HTTP request is available via internal APIs: 
`SolrQueryRequest.get
 
 * PreAnalyzedField and PreAnalyzedUpdateProcessor have been removed due to 
incompatibility with Lucene 10 (SOLR-17839).
 
+* BlobHandler and the `.system` collection have been removed in favour of 
FileStore API.  (SOLR-17851). 
+
 
 === Security
 
diff --git 
a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java 
b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
index a7c6043fb19..74a8e5dd869 100644
--- 
a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
+++ 
b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
@@ -31,9 +31,6 @@ public interface CollectionAdminParams {
 
   String ROLE = "role";
 
-  /** Predefined system collection name. */
-  String SYSTEM_COLL = ".system";
-
   /**
    * A parameter to specify list of Solr nodes to be used (e.g. for collection 
creation or restore
    * operation).
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 4f4aafdaf5b..e42dc929822 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
@@ -279,9 +279,6 @@ public interface CommonParams {
   String TRUE = Boolean.TRUE.toString();
   String FALSE = Boolean.FALSE.toString();
 
-  /** document type in {@link CollectionAdminParams#SYSTEM_COLL} collection. * 
*/
-  String TYPE = "type";
-
   /**
    * Used as a local parameter on queries. cache=false means don't check any 
query or filter caches.
    * cache=true is the default.

Reply via email to