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/17 18:22:51 UTC

[solr] branch main updated: SOLR-16470: Add v2 API for /replication?command=indexversion (#1620)

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 4adf5ac5155 SOLR-16470: Add v2 API for /replication?command=indexversion (#1620)
4adf5ac5155 is described below

commit 4adf5ac5155082e2c64d51fbfa6a01f1a1390858
Author: Matthew Biscocho <54...@users.noreply.github.com>
AuthorDate: Wed May 17 14:22:45 2023 -0400

    SOLR-16470: Add v2 API for /replication?command=indexversion (#1620)
    
    No v2 equivalent existed prior to this commit.  The new v2 API is
    `GET /api/cores/cName/replication/indexversion`.
    
    ---------
    
    Co-authored-by: Jason Gerlowski <ge...@apache.org>
---
 solr/CHANGES.txt                                   |  3 +
 .../apache/solr/handler/ReplicationHandler.java    | 75 +++++++++++++--------
 .../solr/handler/admin/api/CoreReplicationAPI.java | 75 +++++++++++++++++++++
 .../solr/handler/admin/api/ReplicationAPIBase.java | 47 +++++++++++++
 .../handler/admin/api/CoreReplicationAPITest.java  | 76 ++++++++++++++++++++++
 .../pages/user-managed-index-replication.adoc      | 17 +++++
 6 files changed, 266 insertions(+), 27 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 74daffedb05..1802555d81e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -136,6 +136,9 @@ Improvements
   recency now uses `DELETE /api/backups/bName/versions?retainLatest=3`, and index-file "garbage collection" now uses
   `PUT /api/backups/backupName/purgeUnused` (Jason Gerlowski)
 
+* SOLR-16470: `/coreName/replication?commit=indexversion` now has a v2 equivalent, available at
+  `GET /api/cores/coreName/replication/indexversion` (Matthew Biscocho via Jason Gerlowski)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index ae0cd65b6d4..3e358fd8fb7 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -68,6 +68,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RateLimiter;
+import org.apache.solr.api.JerseyResource;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
@@ -91,6 +92,9 @@ import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
 import org.apache.solr.handler.IndexFetcher.IndexFetchResult;
+import org.apache.solr.handler.admin.api.CoreReplicationAPI;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.SolrJerseyResponse;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.request.SolrQueryRequest;
@@ -263,33 +267,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     // This command does not give the current index version of the leader
     // It gives the current 'replicateable' index version
     if (command.equals(CMD_INDEX_VERSION)) {
-      IndexCommit commitPoint = indexCommitPoint; // make a copy so it won't change
-
-      if (commitPoint == null) {
-        // if this handler is 'lazy', we may not have tracked the last commit
-        // because our commit listener is registered on inform
-        commitPoint = core.getDeletionPolicy().getLatestCommit();
-      }
-
-      if (commitPoint != null && replicationEnabled.get()) {
-        //
-        // There is a race condition here.  The commit point may be changed / deleted by the time
-        // we get around to reserving it.  This is a very small window though, and should not result
-        // in a catastrophic failure, but will result in the client getting an empty file list for
-        // the CMD_GET_FILE_LIST command.
-        //
-        core.getDeletionPolicy()
-            .setReserveDuration(commitPoint.getGeneration(), reserveCommitDuration);
-        rsp.add(CMD_INDEX_VERSION, IndexDeletionPolicyWrapper.getCommitTimestamp(commitPoint));
-        rsp.add(GENERATION, commitPoint.getGeneration());
-        rsp.add(STATUS, OK_STATUS);
-      } else {
-        // This happens when replication is not configured to happen after startup and no
-        // commit/optimize has happened yet.
-        rsp.add(CMD_INDEX_VERSION, 0L);
-        rsp.add(GENERATION, 0L);
-        rsp.add(STATUS, OK_STATUS);
-      }
+      final SolrJerseyResponse indexVersionResponse = getIndexVersionResponse();
+      V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, indexVersionResponse);
     } else if (command.equals(CMD_GET_FILE)) {
       getFileStream(solrParams, rsp);
     } else if (command.equals(CMD_GET_FILE_LIST)) {
@@ -798,6 +777,38 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
+  public CoreReplicationAPI.IndexVersionResponse getIndexVersionResponse() throws IOException {
+
+    IndexCommit commitPoint = indexCommitPoint; // make a copy so it won't change
+    CoreReplicationAPI.IndexVersionResponse rsp = new CoreReplicationAPI.IndexVersionResponse();
+    if (commitPoint == null) {
+      // if this handler is 'lazy', we may not have tracked the last commit
+      // because our commit listener is registered on inform
+      commitPoint = core.getDeletionPolicy().getLatestCommit();
+    }
+
+    if (commitPoint != null && replicationEnabled.get()) {
+      //
+      // There is a race condition here.  The commit point may be changed / deleted by the time
+      // we get around to reserving it.  This is a very small window though, and should not result
+      // in a catastrophic failure, but will result in the client getting an empty file list for
+      // the CMD_GET_FILE_LIST command.
+      //
+      core.getDeletionPolicy()
+          .setReserveDuration(commitPoint.getGeneration(), reserveCommitDuration);
+      rsp.indexVersion = IndexDeletionPolicyWrapper.getCommitTimestamp(commitPoint);
+      rsp.generation = commitPoint.getGeneration();
+    } else {
+      // This happens when replication is not configured to happen after startup and no
+      // commit/optimize has happened yet.
+      rsp.indexVersion = 0L;
+      rsp.generation = 0L;
+    }
+    rsp.status = OK_STATUS;
+
+    return rsp;
+  }
+
   /**
    * Retrieves the maximum version number from an index commit. NOTE: The commit <b>MUST</b> be
    * reserved before calling this method
@@ -1464,6 +1475,16 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     log.info("Commits will be reserved for {} ms", reserveCommitDuration);
   }
 
+  @Override
+  public Collection<Class<? extends JerseyResource>> getJerseyResources() {
+    return List.of(CoreReplicationAPI.class);
+  }
+
+  @Override
+  public Boolean registerV2() {
+    return Boolean.TRUE;
+  }
+
   // check leader or follower is enabled
   private boolean isEnabled(NamedList<?> params) {
     if (params == null) return false;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java
new file mode 100644
index 00000000000..29ead1bd519
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java
@@ -0,0 +1,75 @@
+/*
+ * 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 static org.apache.solr.security.PermissionNameProvider.Name.CORE_READ_PERM;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.io.IOException;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/**
+ * V2 APIs for inspecting and replicating indices
+ *
+ * <p>These APIs are analogous to the v1 /coreName/replication APIs.
+ */
+@Path("/cores/{coreName}/replication")
+public class CoreReplicationAPI extends ReplicationAPIBase {
+
+  @Inject
+  public CoreReplicationAPI(SolrCore solrCore, SolrQueryRequest req, SolrQueryResponse rsp) {
+    super(solrCore, req, rsp);
+  }
+
+  @GET
+  @Path("/indexversion")
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(CORE_READ_PERM)
+  public IndexVersionResponse fetchIndexVersion() throws IOException {
+    return doFetchIndexVersion();
+  }
+
+  /** Response for {@link CoreReplicationAPI#fetchIndexVersion()}. */
+  public static class IndexVersionResponse extends SolrJerseyResponse {
+
+    @JsonProperty("indexversion")
+    public Long indexVersion;
+
+    @JsonProperty("generation")
+    public Long generation;
+
+    @JsonProperty("status")
+    public String status;
+
+    public IndexVersionResponse() {}
+
+    public IndexVersionResponse(Long indexVersion, Long generation, String status) {
+      this.indexVersion = indexVersion;
+      this.generation = generation;
+      this.status = status;
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java
new file mode 100644
index 00000000000..e53261350ff
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java
@@ -0,0 +1,47 @@
+/*
+ * 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 java.io.IOException;
+import org.apache.solr.api.JerseyResource;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/** A common parent for "replication" (i.e. replication-level) APIs. */
+public abstract class ReplicationAPIBase extends JerseyResource {
+
+  protected final SolrCore solrCore;
+  protected final SolrQueryRequest solrQueryRequest;
+  protected final SolrQueryResponse solrQueryResponse;
+
+  public ReplicationAPIBase(
+      SolrCore solrCore, SolrQueryRequest solrQueryRequest, SolrQueryResponse solrQueryResponse) {
+    this.solrCore = solrCore;
+    this.solrQueryRequest = solrQueryRequest;
+    this.solrQueryResponse = solrQueryResponse;
+  }
+
+  protected CoreReplicationAPI.IndexVersionResponse doFetchIndexVersion() throws IOException {
+
+    ReplicationHandler replicationHandler =
+        (ReplicationHandler) solrCore.getRequestHandler(ReplicationHandler.PATH);
+
+    return replicationHandler.getIndexVersionResponse();
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java
new file mode 100644
index 00000000000..98961280fa1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java
@@ -0,0 +1,76 @@
+/*
+ * 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 io.opentracing.noop.NoopSpan;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Unit tests for {@link CoreReplicationAPI} */
+public class CoreReplicationAPITest extends SolrTestCaseJ4 {
+
+  private CoreReplicationAPI coreReplicationAPI;
+  private SolrCore mockCore;
+  private ReplicationHandler mockReplicationHandler;
+  private static final String coreName = "test";
+  private SolrQueryRequest mockQueryRequest;
+  private SolrQueryResponse queryResponse;
+
+  @BeforeClass
+  public static void ensureWorkingMockito() {
+    assumeWorkingMockito();
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    setUpMocks();
+    mockQueryRequest = mock(SolrQueryRequest.class);
+    when(mockQueryRequest.getSpan()).thenReturn(NoopSpan.INSTANCE);
+    queryResponse = new SolrQueryResponse();
+    coreReplicationAPI = new CoreReplicationAPI(mockCore, mockQueryRequest, queryResponse);
+  }
+
+  @Test
+  public void testGetIndexVersion() throws Exception {
+    CoreReplicationAPI.IndexVersionResponse expected =
+        new CoreReplicationAPI.IndexVersionResponse(123L, 123L, "testGeneration");
+    when(mockReplicationHandler.getIndexVersionResponse()).thenReturn(expected);
+
+    CoreReplicationAPI.IndexVersionResponse response = coreReplicationAPI.doFetchIndexVersion();
+    assertEquals(expected.indexVersion, response.indexVersion);
+    assertEquals(expected.generation, response.generation);
+    assertEquals(expected.status, response.status);
+  }
+
+  private void setUpMocks() {
+    mockCore = mock(SolrCore.class);
+    mockReplicationHandler = mock(ReplicationHandler.class);
+    when(mockCore.getRequestHandler(ReplicationHandler.PATH)).thenReturn(mockReplicationHandler);
+  }
+}
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
index b09be99ca8c..4def27755a2 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
@@ -378,9 +378,26 @@ http://_leader_host:port_/solr/_core_name_/replication?command=disablereplicatio
 `indexversion`::
 Return the version of the latest replicatable index on the specified leader or follower.
 +
+====
+[.tab-label]*V1 API*
+
 [source,bash]
+----
 http://_host:port_/solr/_core_name_/replication?command=indexversion
 
+----
+====
++
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+http://_host:port_/api/cores/_core_name_/replication/indexversion
+
+----
+====
+
 `fetchindex`::
 Force the specified follower to fetch a copy of the index from its leader.
 +