You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2023/05/30 15:28:50 UTC

[solr] branch main updated: SOLR-16395 JAX-RS conversion for /schema, uniqueKey, similarity and version (#1649)

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

gerlowskija 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 95a1046d123 SOLR-16395 JAX-RS conversion for /schema, uniqueKey, similarity and version (#1649)
95a1046d123 is described below

commit 95a1046d123b688b1c21dd932b352d97925495fc
Author: bszabo97 <79...@users.noreply.github.com>
AuthorDate: Tue May 30 17:28:42 2023 +0200

    SOLR-16395 JAX-RS conversion for /schema, uniqueKey, similarity and version (#1649)
    
    Migrates several v2 `GET /schema` APIs from the legacy annotation framework
    to JAX-RS, including:
    
    - `/schema` - fetch the whole schema
    - `/schema/uniquekey` - fetch the name of the schemas unique key
    - `/schema/similarity` - fetch the similarity implementation
    - `/schema/version` - fetch the schema version
    
    This change doesn't modify the APIs themselves, so users should remain unaffected.
    
    ---------
    
    Co-authored-by: Jason Gerlowski <ge...@apache.org>
---
 .../org/apache/solr/handler/SchemaHandler.java     |  41 +++----
 .../solr/handler/admin/api/GetSchemaAPI.java       | 120 +++++++++++++++++++++
 .../solr/handler/admin/api/SchemaInfoAPI.java      |  48 ---------
 .../handler/admin/api/SchemaSimilarityAPI.java     |  48 ---------
 .../solr/handler/admin/api/SchemaUniqueKeyAPI.java |  48 ---------
 .../solr/handler/admin/api/SchemaVersionAPI.java   |  48 ---------
 .../org/apache/solr/jersey/InjectionFactories.java |  19 ++++
 .../org/apache/solr/jersey/JerseyApplications.java |   4 +
 .../solr/handler/admin/TestApiFramework.java       |   1 -
 .../solr/handler/admin/api/GetSchemaAPITest.java   |  93 ++++++++++++++++
 .../handler/admin/api/V2SchemaAPIMappingTest.java  |   8 --
 11 files changed, 258 insertions(+), 220 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index da216ed6510..89dc0ffd3de 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -47,19 +47,16 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.admin.api.GetSchemaAPI;
 import org.apache.solr.handler.admin.api.SchemaBulkModifyAPI;
 import org.apache.solr.handler.admin.api.SchemaGetDynamicFieldAPI;
 import org.apache.solr.handler.admin.api.SchemaGetFieldAPI;
 import org.apache.solr.handler.admin.api.SchemaGetFieldTypeAPI;
-import org.apache.solr.handler.admin.api.SchemaInfoAPI;
 import org.apache.solr.handler.admin.api.SchemaListAllCopyFieldsAPI;
 import org.apache.solr.handler.admin.api.SchemaListAllDynamicFieldsAPI;
 import org.apache.solr.handler.admin.api.SchemaListAllFieldTypesAPI;
 import org.apache.solr.handler.admin.api.SchemaListAllFieldsAPI;
 import org.apache.solr.handler.admin.api.SchemaNameAPI;
-import org.apache.solr.handler.admin.api.SchemaSimilarityAPI;
-import org.apache.solr.handler.admin.api.SchemaUniqueKeyAPI;
-import org.apache.solr.handler.admin.api.SchemaVersionAPI;
 import org.apache.solr.handler.admin.api.SchemaZkVersionAPI;
 import org.apache.solr.handler.api.V2ApiUtils;
 import org.apache.solr.pkg.PackageListeningClassLoader;
@@ -140,19 +137,29 @@ public class SchemaHandler extends RequestHandlerBase
       String path = (String) req.getContext().get("path");
       switch (path) {
         case "/schema":
-          rsp.add(IndexSchema.SCHEMA, req.getSchema().getNamedPropertyValues());
-          break;
+          {
+            V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+                rsp, new GetSchemaAPI(req.getCore().getLatestSchema()).getSchemaInfo());
+            break;
+          }
         case "/schema/version":
-          rsp.add(IndexSchema.VERSION, req.getSchema().getVersion());
-          break;
+          {
+            V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+                rsp, new GetSchemaAPI(req.getCore().getLatestSchema()).getSchemaVersion());
+            break;
+          }
         case "/schema/uniquekey":
-          rsp.add(IndexSchema.UNIQUE_KEY, req.getSchema().getUniqueKeyField().getName());
-          break;
+          {
+            V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+                rsp, new GetSchemaAPI(req.getCore().getLatestSchema()).getSchemaUniqueKey());
+            break;
+          }
         case "/schema/similarity":
-          rsp.add(
-              IndexSchema.SIMILARITY,
-              req.getSchema().getSimilarityFactory().getNamedPropertyValues());
-          break;
+          {
+            V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+                rsp, new GetSchemaAPI(req.getCore().getLatestSchema()).getSchemaSimilarity());
+            break;
+          }
         case "/schema/name":
           {
             V2ApiUtils.squashIntoSolrResponseWithoutHeader(
@@ -313,10 +320,6 @@ public class SchemaHandler extends RequestHandlerBase
   public Collection<Api> getApis() {
 
     final List<Api> apis = new ArrayList<>();
-    apis.addAll(AnnotatedApi.getApis(new SchemaInfoAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new SchemaUniqueKeyAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new SchemaVersionAPI(this)));
-    apis.addAll(AnnotatedApi.getApis(new SchemaSimilarityAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SchemaZkVersionAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SchemaListAllFieldsAPI(this)));
     apis.addAll(AnnotatedApi.getApis(new SchemaGetFieldAPI(this)));
@@ -332,7 +335,7 @@ public class SchemaHandler extends RequestHandlerBase
 
   @Override
   public Collection<Class<? extends JerseyResource>> getJerseyResources() {
-    return List.of(SchemaNameAPI.class);
+    return List.of(SchemaNameAPI.class, GetSchemaAPI.class);
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/GetSchemaAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/GetSchemaAPI.java
new file mode 100644
index 00000000000..461120aab15
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/GetSchemaAPI.java
@@ -0,0 +1,120 @@
+/*
+ * 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.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Map;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import org.apache.solr.api.JerseyResource;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.security.PermissionNameProvider;
+
+@Path("/{a:cores|collections}/{collectionName}/schema")
+public class GetSchemaAPI extends JerseyResource {
+
+  private IndexSchema indexSchema;
+
+  @Inject
+  public GetSchemaAPI(IndexSchema indexSchema) {
+    this.indexSchema = indexSchema;
+  }
+
+  @GET
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(PermissionNameProvider.Name.SCHEMA_READ_PERM)
+  public SchemaInfoResponse getSchemaInfo() {
+    final var response = instantiateJerseyResponse(SchemaInfoResponse.class);
+
+    response.schema = indexSchema.getNamedPropertyValues();
+
+    return response;
+  }
+
+  public static class SchemaInfoResponse extends SolrJerseyResponse {
+    // TODO The schema response is quite complicated, so for the moment it's sufficient to record it
+    // here only as a Map.  However, if SOLR-16825 is tackled then there will be a lot of value in
+    // describing this response format more accurately so that clients can navigate the contents
+    // without lots of map fetching and casting.
+    @JsonProperty("schema")
+    public Map<String, Object> schema;
+  }
+
+  @GET
+  @Path("/similarity")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(PermissionNameProvider.Name.SCHEMA_READ_PERM)
+  public SchemaSimilarityResponse getSchemaSimilarity() {
+    final var response = instantiateJerseyResponse(SchemaSimilarityResponse.class);
+
+    response.similarity = indexSchema.getSimilarityFactory().getNamedPropertyValues();
+
+    return response;
+  }
+
+  public static class SchemaSimilarityResponse extends SolrJerseyResponse {
+    // TODO The schema response is quite complicated, so for the moment it's sufficient to record it
+    // here only as a Map.  However, if SOLR-16825 is tackled then there will be a lot of value in
+    // describing this response format more accurately so that clients can navigate the contents
+    // without lots of map fetching and casting.
+    @JsonProperty("similarity")
+    public SimpleOrderedMap<Object> similarity;
+  }
+
+  @GET
+  @Path("/uniquekey")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(PermissionNameProvider.Name.SCHEMA_READ_PERM)
+  public SchemaUniqueKeyResponse getSchemaUniqueKey() {
+    final var response = instantiateJerseyResponse(SchemaUniqueKeyResponse.class);
+
+    response.uniqueKey = indexSchema.getUniqueKeyField().getName();
+
+    return response;
+  }
+
+  public static class SchemaUniqueKeyResponse extends SolrJerseyResponse {
+    @JsonProperty("uniqueKey")
+    public String uniqueKey;
+  }
+
+  @GET
+  @Path("/version")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(PermissionNameProvider.Name.SCHEMA_READ_PERM)
+  public SchemaVersionResponse getSchemaVersion() {
+    final var response = instantiateJerseyResponse(SchemaVersionResponse.class);
+
+    response.version = indexSchema.getVersion();
+
+    return response;
+  }
+
+  public static class SchemaVersionResponse extends SolrJerseyResponse {
+    @JsonProperty("version")
+    public float version;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaInfoAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaInfoAPI.java
deleted file mode 100644
index 9ac6e3a52d9..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaInfoAPI.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.GET;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.SchemaHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.PermissionNameProvider;
-
-/**
- * V2 API for getting basic information about an in-use schema
- *
- * <p>This API (GET /v2/collections/collectionName/schema) is analogous to the v1
- * /solr/collectionName/schema API.
- */
-public class SchemaInfoAPI {
-  private final SchemaHandler schemaHandler;
-
-  public SchemaInfoAPI(SchemaHandler schemaHandler) {
-    this.schemaHandler = schemaHandler;
-  }
-
-  @EndPoint(
-      path = {"/schema"},
-      method = GET,
-      permission = PermissionNameProvider.Name.SCHEMA_READ_PERM)
-  public void getSchemaInfo(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    schemaHandler.handleRequestBody(req, rsp);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaSimilarityAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaSimilarityAPI.java
deleted file mode 100644
index 5e492646d84..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaSimilarityAPI.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.GET;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.SchemaHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.PermissionNameProvider;
-
-/**
- * V2 API for getting information about the 'similarity' settings for an in-use schema.
- *
- * <p>This API (GET /v2/collections/collectionName/schema/similarity) is analogous to the v1
- * /solr/collectionName/schema/similarity API.
- */
-public class SchemaSimilarityAPI {
-  private final SchemaHandler schemaHandler;
-
-  public SchemaSimilarityAPI(SchemaHandler schemaHandler) {
-    this.schemaHandler = schemaHandler;
-  }
-
-  @EndPoint(
-      path = {"/schema/similarity"},
-      method = GET,
-      permission = PermissionNameProvider.Name.SCHEMA_READ_PERM)
-  public void getSchemaSimilarity(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    schemaHandler.handleRequestBody(req, rsp);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaUniqueKeyAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaUniqueKeyAPI.java
deleted file mode 100644
index 13cb8490f5a..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaUniqueKeyAPI.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.GET;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.SchemaHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.PermissionNameProvider;
-
-/**
- * V2 API for getting the name of the unique-key field for an in-use schema.
- *
- * <p>This API (GET /v2/collections/collectionName/schema/uniquekey) is analogous to the v1
- * /solr/collectionName/schema/uniquekey API.
- */
-public class SchemaUniqueKeyAPI {
-  private final SchemaHandler schemaHandler;
-
-  public SchemaUniqueKeyAPI(SchemaHandler schemaHandler) {
-    this.schemaHandler = schemaHandler;
-  }
-
-  @EndPoint(
-      path = {"/schema/uniquekey"},
-      method = GET,
-      permission = PermissionNameProvider.Name.SCHEMA_READ_PERM)
-  public void getSchemaUniqueKey(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    schemaHandler.handleRequestBody(req, rsp);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaVersionAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaVersionAPI.java
deleted file mode 100644
index 6d3538f5c77..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/SchemaVersionAPI.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.GET;
-
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.handler.SchemaHandler;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.PermissionNameProvider;
-
-/**
- * V2 API for getting the version of an in-use schema.
- *
- * <p>This API (GET /v2/collections/collectionName/schema/version) is analogous to the v1
- * /solr/collectionName/schema/version API.
- */
-public class SchemaVersionAPI {
-  private final SchemaHandler schemaHandler;
-
-  public SchemaVersionAPI(SchemaHandler schemaHandler) {
-    this.schemaHandler = schemaHandler;
-  }
-
-  @EndPoint(
-      path = {"/schema/version"},
-      method = GET,
-      permission = PermissionNameProvider.Name.SCHEMA_READ_PERM)
-  public void getSchemaVersion(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    schemaHandler.handleRequestBody(req, rsp);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java b/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
index 2cbc5f99cfd..d6111b15d19 100644
--- a/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
+++ b/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
@@ -24,6 +24,7 @@ import javax.ws.rs.container.ContainerRequestContext;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
 import org.glassfish.hk2.api.Factory;
 
 public class InjectionFactories {
@@ -85,6 +86,24 @@ public class InjectionFactories {
     public void dispose(SolrCore instance) {}
   }
 
+  public static class ReuseFromContextIndexSchemaFactory implements Factory<IndexSchema> {
+
+    private final SolrCore solrCore;
+
+    @Inject
+    public ReuseFromContextIndexSchemaFactory(SolrCore solrCore) {
+      this.solrCore = solrCore;
+    }
+
+    @Override
+    public IndexSchema provide() {
+      return solrCore.getLatestSchema();
+    }
+
+    @Override
+    public void dispose(IndexSchema instance) {}
+  }
+
   public static class SingletonFactory<T> implements Factory<T> {
 
     private final T singletonVal;
diff --git a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
index ef9587ceb90..d8e3568e540 100644
--- a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
+++ b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.SolrVersion;
 import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJsonProvider;
@@ -116,6 +117,9 @@ public class JerseyApplications {
               bindFactory(InjectionFactories.ReuseFromContextSolrCoreFactory.class)
                   .to(SolrCore.class)
                   .in(RequestScoped.class);
+              bindFactory(InjectionFactories.ReuseFromContextIndexSchemaFactory.class)
+                  .to(IndexSchema.class)
+                  .in(RequestScoped.class);
             }
           });
     }
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 a04e4bc4aef..07601cc624d 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
@@ -138,7 +138,6 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     methodNames.add(rsp.getValues()._getStr("/spec[0]/methods[0]", null));
     methodNames.add(rsp.getValues()._getStr("/spec[1]/methods[0]", null));
     assertTrue(methodNames.contains("POST"));
-    assertTrue(methodNames.contains("GET"));
   }
 
   public void testPayload() {
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/GetSchemaAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/GetSchemaAPITest.java
new file mode 100644
index 00000000000..5aa3e3b8890
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/GetSchemaAPITest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Map;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.SimilarityFactory;
+import org.apache.solr.schema.StrField;
+import org.junit.Before;
+import org.junit.Test;
+
+/** Unit tests for {@link GetSchemaAPI} */
+public class GetSchemaAPITest extends SolrTestCaseJ4 {
+
+  private IndexSchema mockSchema;
+  private GetSchemaAPI api;
+
+  @Before
+  public void setUpMocks() {
+    assumeWorkingMockito();
+
+    mockSchema = mock(IndexSchema.class);
+    api = new GetSchemaAPI(mockSchema);
+  }
+
+  @Test
+  public void testReliesOnIndexSchemaWhenFetchingWholeSchema() {
+    when(mockSchema.getNamedPropertyValues()).thenReturn(Map.of("flagKey", "flagValue"));
+
+    final var response = api.getSchemaInfo();
+
+    assertNotNull(response);
+    assertNotNull(response.schema);
+    assertEquals(1, response.schema.size());
+    assertEquals("flagValue", response.schema.get("flagKey"));
+  }
+
+  @Test
+  public void testReliesOnIndexSchemaWhenFetchingSimilarity() {
+    final var map = new SimpleOrderedMap<Object>();
+    map.add("flagKey", "flagValue");
+    final SimilarityFactory mockSimFactory = mock(SimilarityFactory.class);
+    when(mockSimFactory.getNamedPropertyValues()).thenReturn(map);
+    when(mockSchema.getSimilarityFactory()).thenReturn(mockSimFactory);
+
+    final var response = api.getSchemaSimilarity();
+
+    assertNotNull(response);
+    assertNotNull(response.similarity);
+    assertEquals(1, response.similarity.size());
+    assertEquals("flagValue", response.similarity.get("flagKey"));
+  }
+
+  @Test
+  public void testReliesOnIndexSchemaWhenFetchingUniqueKey() {
+    when(mockSchema.getUniqueKeyField()).thenReturn(new SchemaField("myUniqueKey", new StrField()));
+
+    final var response = api.getSchemaUniqueKey();
+
+    assertNotNull(response);
+    assertEquals("myUniqueKey", response.uniqueKey);
+  }
+
+  @Test
+  public void testReliesOnIndexSchemaWhenFetchingVersion() {
+    when(mockSchema.getVersion()).thenReturn(123.456f);
+
+    final var response = api.getSchemaVersion();
+
+    assertNotNull(response);
+    assertEquals(123.456f, response.version, 0.1f);
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/V2SchemaAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/V2SchemaAPIMappingTest.java
index 7de25940707..cb34f8a482e 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/api/V2SchemaAPIMappingTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/V2SchemaAPIMappingTest.java
@@ -26,10 +26,6 @@ public class V2SchemaAPIMappingTest extends V2ApiMappingTest<SchemaHandler> {
 
   @Override
   public void populateApiBag() {
-    apiBag.registerObject(new SchemaInfoAPI(getRequestHandler()));
-    apiBag.registerObject(new SchemaUniqueKeyAPI(getRequestHandler()));
-    apiBag.registerObject(new SchemaVersionAPI(getRequestHandler()));
-    apiBag.registerObject(new SchemaSimilarityAPI(getRequestHandler()));
     apiBag.registerObject(new SchemaZkVersionAPI(getRequestHandler()));
     apiBag.registerObject(new SchemaListAllFieldsAPI(getRequestHandler()));
     apiBag.registerObject(new SchemaGetFieldAPI(getRequestHandler()));
@@ -53,7 +49,6 @@ public class V2SchemaAPIMappingTest extends V2ApiMappingTest<SchemaHandler> {
 
   @Test
   public void testGetSchemaInfoApis() {
-    assertAnnotatedApiExistsFor("GET", "/schema");
     assertAnnotatedApiExistsFor("GET", "/schema/dynamicfields");
     assertAnnotatedApiExistsFor("GET", "/schema/dynamicfields/someDynamicField");
     assertAnnotatedApiExistsFor("GET", "/schema/fieldtypes");
@@ -61,9 +56,6 @@ public class V2SchemaAPIMappingTest extends V2ApiMappingTest<SchemaHandler> {
     assertAnnotatedApiExistsFor("GET", "/schema/fields");
     assertAnnotatedApiExistsFor("GET", "/schema/fields/someField");
     assertAnnotatedApiExistsFor("GET", "/schema/copyfields");
-    assertAnnotatedApiExistsFor("GET", "/schema/similarity");
-    assertAnnotatedApiExistsFor("GET", "/schema/uniquekey");
-    assertAnnotatedApiExistsFor("GET", "/schema/version");
     assertAnnotatedApiExistsFor("GET", "/schema/zkversion");
   }