You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2020/08/28 20:43:31 UTC
[lucene-solr] branch master updated: SOLR-14616: Remove CDCR
This is an automated email from the ASF dual-hosted git repository.
ishan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
The following commit(s) were added to refs/heads/master by this push:
new d84977e SOLR-14616: Remove CDCR
d84977e is described below
commit d84977eb5cde00f0e92f71837bdf9cee25e0b54a
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Sat Aug 29 02:13:13 2020 +0530
SOLR-14616: Remove CDCR
---
solr/CHANGES.txt | 2 +
.../org/apache/solr/cloud/RecoveryStrategy.java | 7 -
.../org/apache/solr/handler/CdcrBufferManager.java | 71 --
.../solr/handler/CdcrBufferStateManager.java | 178 ----
.../solr/handler/CdcrLeaderStateManager.java | 162 ----
.../java/org/apache/solr/handler/CdcrParams.java | 256 ------
.../solr/handler/CdcrProcessStateManager.java | 178 ----
.../org/apache/solr/handler/CdcrReplicator.java | 258 ------
.../apache/solr/handler/CdcrReplicatorManager.java | 441 ----------
.../solr/handler/CdcrReplicatorScheduler.java | 116 ---
.../apache/solr/handler/CdcrReplicatorState.java | 299 -------
.../apache/solr/handler/CdcrRequestHandler.java | 880 --------------------
.../org/apache/solr/handler/CdcrStateManager.java | 47 --
.../solr/handler/CdcrUpdateLogSynchronizer.java | 192 -----
.../java/org/apache/solr/handler/IndexFetcher.java | 124 +--
.../apache/solr/handler/ReplicationHandler.java | 39 -
.../handler/component/RealTimeGetComponent.java | 15 +-
.../org/apache/solr/update/CdcrTransactionLog.java | 401 ---------
.../java/org/apache/solr/update/CdcrUpdateLog.java | 796 ------------------
.../apache/solr/update/DefaultSolrCoreState.java | 41 -
.../java/org/apache/solr/update/SolrCoreState.java | 17 -
.../src/java/org/apache/solr/update/UpdateLog.java | 3 +-
.../solr/update/processor/CdcrUpdateProcessor.java | 132 ---
.../processor/CdcrUpdateProcessorFactory.java | 46 --
.../solr/collection1/conf/solrconfig-cdcr.xml | 77 --
.../collection1/conf/solrconfig-cdcrupdatelog.xml | 49 --
.../configsets/cdcr-cluster1/conf/managed-schema | 29 -
.../configsets/cdcr-cluster1/conf/solrconfig.xml | 80 --
.../configsets/cdcr-cluster2/conf/managed-schema | 29 -
.../configsets/cdcr-cluster2/conf/solrconfig.xml | 80 --
.../cdcr-source-disabled/conf/schema.xml | 29 -
.../cdcr-source-disabled/conf/solrconfig.xml | 60 --
.../solr/configsets/cdcr-source/conf/schema.xml | 29 -
.../configsets/cdcr-source/conf/solrconfig.xml | 75 --
.../solr/configsets/cdcr-target/conf/schema.xml | 29 -
.../configsets/cdcr-target/conf/solrconfig.xml | 62 --
.../solr/cloud/cdcr/BaseCdcrDistributedZkTest.java | 906 ---------------------
.../solr/cloud/cdcr/CdcrBidirectionalTest.java | 244 ------
.../apache/solr/cloud/cdcr/CdcrBootstrapTest.java | 373 ---------
.../solr/cloud/cdcr/CdcrOpsAndBoundariesTest.java | 332 --------
.../cloud/cdcr/CdcrReplicationHandlerTest.java | 332 --------
.../solr/cloud/cdcr/CdcrRequestHandlerTest.java | 183 -----
.../org/apache/solr/cloud/cdcr/CdcrTestsUtil.java | 274 -------
.../cloud/cdcr/CdcrVersionReplicationTest.java | 307 -------
.../solr/cloud/cdcr/CdcrWithNodesRestartsTest.java | 359 --------
.../org/apache/solr/search/TestRealTimeGet.java | 1 -
.../test/org/apache/solr/search/TestRecovery.java | 1 -
.../org/apache/solr/search/TestStressRecovery.java | 1 -
.../org/apache/solr/update/CdcrUpdateLogTest.java | 783 ------------------
.../solr/update/TestInPlaceUpdatesDistrib.java | 2 -
solr/solr-ref-guide/src/aliases.adoc | 2 -
solr/solr-ref-guide/src/cdcr-api.adoc | 321 --------
solr/solr-ref-guide/src/cdcr-architecture.adoc | 167 ----
solr/solr-ref-guide/src/cdcr-config.adoc | 376 ---------
solr/solr-ref-guide/src/cdcr-operations.adoc | 49 --
.../src/cross-data-center-replication-cdcr.adoc | 63 --
.../src/major-changes-from-solr-5-to-solr-6.adoc | 2 +-
solr/solr-ref-guide/src/solrcloud.adoc | 5 +-
.../src/java/org/apache/solr/SolrTestCaseJ4.java | 14 -
59 files changed, 11 insertions(+), 10415 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 018d893..d376597 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -48,6 +48,8 @@ Other Changes
----------------------
* SOLR-14656: Autoscaling framework removed (Ishan Chattopadhyaya, noble, Ilan Ginzburg)
+* SOLR-14616: CDCR support removed (Ishan Chattopadhyaya)
+
* LUCENE-9391: Upgrade HPPC to 0.8.2. (Haoyu Zhai)
* SOLR-10288: Remove non-minified JavaScript from the webapp. (Erik Hatcher, marcussorealheis)
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index c2db3f4..a3e2f7e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -58,7 +58,6 @@ import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.PeerSyncWithLeader;
import org.apache.solr.update.UpdateLog;
@@ -241,12 +240,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
ModifiableSolrParams solrParams = new ModifiableSolrParams();
solrParams.set(ReplicationHandler.LEADER_URL, leaderUrl);
solrParams.set(ReplicationHandler.SKIP_COMMIT_ON_LEADER_VERSION_ZERO, replicaType == Replica.Type.TLOG);
- // always download the tlogs from the leader when running with cdcr enabled. We need to have all the tlogs
- // to ensure leader failover doesn't cause missing docs on the target
- if (core.getUpdateHandler().getUpdateLog() != null
- && core.getUpdateHandler().getUpdateLog() instanceof CdcrUpdateLog) {
- solrParams.set(ReplicationHandler.TLOG_FILES, true);
- }
if (isClosed()) return; // we check closed on return
boolean success = replicationHandler.doFetch(solrParams, false).getSuccessful();
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrBufferManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrBufferManager.java
deleted file mode 100644
index 8696379..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrBufferManager.java
+++ /dev/null
@@ -1,71 +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.core.SolrCore;
-import org.apache.solr.update.CdcrUpdateLog;
-
-/**
- * This manager is responsible in enabling or disabling the buffering of the update logs. Currently, buffer
- * is always activated for non-leader nodes. For leader nodes, it is enabled only if the user explicitly
- * enabled it with the action {@link org.apache.solr.handler.CdcrParams.CdcrAction#ENABLEBUFFER}.
- */
-class CdcrBufferManager implements CdcrStateManager.CdcrStateObserver {
-
- private CdcrLeaderStateManager leaderStateManager;
- private CdcrBufferStateManager bufferStateManager;
-
- private final SolrCore core;
-
- CdcrBufferManager(SolrCore core) {
- this.core = core;
- }
-
- void setLeaderStateManager(final CdcrLeaderStateManager leaderStateManager) {
- this.leaderStateManager = leaderStateManager;
- this.leaderStateManager.register(this);
- }
-
- void setBufferStateManager(final CdcrBufferStateManager bufferStateManager) {
- this.bufferStateManager = bufferStateManager;
- this.bufferStateManager.register(this);
- }
-
- /**
- * This method is synchronised as it can both be called by the leaderStateManager and the bufferStateManager.
- */
- @Override
- public synchronized void stateUpdate() {
- CdcrUpdateLog ulog = (CdcrUpdateLog) core.getUpdateHandler().getUpdateLog();
-
- // If I am not the leader, I should always buffer my updates
- if (!leaderStateManager.amILeader()) {
- ulog.enableBuffer();
- return;
- }
- // If I am the leader, I should buffer my updates only if buffer is enabled
- else if (bufferStateManager.getState().equals(CdcrParams.BufferState.ENABLED)) {
- ulog.enableBuffer();
- return;
- }
-
- // otherwise, disable the buffer
- ulog.disableBuffer();
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
deleted file mode 100644
index 49d19f1..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
+++ /dev/null
@@ -1,178 +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.common.cloud.SolrZkClient;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.core.SolrCore;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-import java.nio.charset.Charset;
-
-/**
- * Manage the state of the update log buffer. It is responsible of synchronising the state
- * through Zookeeper. The state of the buffer is stored in the zk node defined by {@link #getZnodePath()}.
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-class CdcrBufferStateManager extends CdcrStateManager {
-
- private CdcrParams.BufferState state = DEFAULT_STATE;
-
- private BufferStateWatcher wrappedWatcher;
- private Watcher watcher;
-
- private SolrCore core;
-
- static CdcrParams.BufferState DEFAULT_STATE = CdcrParams.BufferState.ENABLED;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrBufferStateManager(final SolrCore core, SolrParams bufferConfiguration) {
- this.core = core;
-
- // Ensure that the state znode exists
- this.createStateNode();
-
- // set default state
- if (bufferConfiguration != null) {
- byte[] defaultState = bufferConfiguration.get(
- CdcrParams.DEFAULT_STATE_PARAM, DEFAULT_STATE.toLower()).getBytes(Charset.forName("UTF-8"));
- state = CdcrParams.BufferState.get(defaultState);
- }
- this.setState(state); // notify observers
-
- // Startup and register the watcher at startup
- try {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- watcher = this.initWatcher(zkClient);
- this.setState(CdcrParams.BufferState.get(zkClient.getData(this.getZnodePath(), watcher, null, true)));
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed fetching initial state", e);
- }
- }
-
- /**
- * SolrZkClient does not guarantee that a watch object will only be triggered once for a given notification
- * if we does not wrap the watcher - see SOLR-6621.
- */
- private Watcher initWatcher(SolrZkClient zkClient) {
- wrappedWatcher = new BufferStateWatcher();
- return zkClient.wrapWatcher(wrappedWatcher);
- }
-
- private String getZnodeBase() {
- return "/collections/" + core.getCoreDescriptor().getCloudDescriptor().getCollectionName() + "/cdcr/state";
- }
-
- private String getZnodePath() {
- return getZnodeBase() + "/buffer";
- }
-
- void setState(CdcrParams.BufferState state) {
- if (this.state != state) {
- this.state = state;
- this.callback(); // notify the observers of a state change
- }
- }
-
- CdcrParams.BufferState getState() {
- return state;
- }
-
- /**
- * Synchronise the state to Zookeeper. This method must be called only by the handler receiving the
- * action.
- */
- void synchronize() {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- zkClient.setData(this.getZnodePath(), this.getState().getBytes(), true);
- // check if nobody changed it in the meantime, and set a new watcher
- this.setState(CdcrParams.BufferState.get(zkClient.getData(this.getZnodePath(), watcher, null, true)));
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed synchronising new state", e);
- }
- }
-
- private void createStateNode() {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- if (!zkClient.exists(this.getZnodePath(), true)) {
- if (!zkClient.exists(this.getZnodeBase(), true)) {
- zkClient.makePath(this.getZnodeBase(), null, CreateMode.PERSISTENT, null, false, true); // Should be a no-op if node exists
- }
- zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
- if (log.isInfoEnabled()) {
- log.info("Created znode {}", this.getZnodePath());
- }
- }
- } catch (KeeperException.NodeExistsException ne) {
- // Someone got in first and created the node.
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed to create CDCR buffer state node", e);
- }
- }
-
- void shutdown() {
- if (wrappedWatcher != null) {
- wrappedWatcher.cancel(); // cancel the watcher to avoid spurious warn messages during shutdown
- }
- }
-
- private class BufferStateWatcher implements Watcher {
-
- private boolean isCancelled = false;
-
- /**
- * Cancel the watcher to avoid spurious warn messages during shutdown.
- */
- void cancel() {
- isCancelled = true;
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (isCancelled) return; // if the watcher is cancelled, do nothing.
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-
- log.info("The CDCR buffer state has changed: {} @ {}:{}", event, collectionName, shard);
- // session events are not change events, and do not remove the watcher
- if (Event.EventType.None.equals(event.getType())) {
- return;
- }
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- CdcrParams.BufferState state = CdcrParams.BufferState.get(zkClient.getData(CdcrBufferStateManager.this.getZnodePath(), watcher, null, true));
- log.info("Received new CDCR buffer state from watcher: {} @ {}:{}", state, collectionName, shard);
- CdcrBufferStateManager.this.setState(state);
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed synchronising new state @ {}:{}", collectionName, shard, e);
- }
- }
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
deleted file mode 100644
index c9bc5fd..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
+++ /dev/null
@@ -1,162 +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 java.lang.invoke.MethodHandles;
-
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.core.SolrCore;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * Manage the leader state of the CDCR nodes.
- * </p>
- * <p>
- * It takes care of notifying the {@link CdcrReplicatorManager} in case
- * of a leader state change.
- * </p>
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-class CdcrLeaderStateManager extends CdcrStateManager {
-
- private boolean amILeader = false;
-
- private LeaderStateWatcher wrappedWatcher;
- private Watcher watcher;
-
- private SolrCore core;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrLeaderStateManager(final SolrCore core) {
- this.core = core;
-
- // Fetch leader state and register the watcher at startup
- try {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- ClusterState clusterState = core.getCoreContainer().getZkController().getClusterState();
-
- watcher = this.initWatcher(zkClient);
- // if the node does not exist, it means that the leader was not yet registered. This can happen
- // when the cluster is starting up. The core is not yet fully loaded, and the leader election process
- // is waiting for it.
- if (this.isLeaderRegistered(zkClient, clusterState)) {
- this.checkIfIAmLeader();
- }
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed fetching initial leader state and setting watch", e);
- }
- }
-
- /**
- * Checks if the leader is registered. If it is not registered, we are probably at the
- * initialisation phase of the cluster. In this case, we must attach a watcher to
- * be notified when the leader is registered.
- */
- private boolean isLeaderRegistered(SolrZkClient zkClient, ClusterState clusterState)
- throws KeeperException, InterruptedException {
- // First check if the znode exists, and register the watcher at the same time
- return zkClient.exists(this.getZnodePath(), watcher, true) != null;
- }
-
- /**
- * SolrZkClient does not guarantee that a watch object will only be triggered once for a given notification
- * if we does not wrap the watcher - see SOLR-6621.
- */
- private Watcher initWatcher(SolrZkClient zkClient) {
- wrappedWatcher = new LeaderStateWatcher();
- return zkClient.wrapWatcher(wrappedWatcher);
- }
-
- private void checkIfIAmLeader() throws KeeperException, InterruptedException {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- ZkNodeProps props = ZkNodeProps.load(zkClient.getData(CdcrLeaderStateManager.this.getZnodePath(), null, null, true));
- if (props != null) {
- CdcrLeaderStateManager.this.setAmILeader(props.get("core").equals(core.getName()));
- }
- }
-
- private String getZnodePath() {
- String myShardId = core.getCoreDescriptor().getCloudDescriptor().getShardId();
- String myCollection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- return "/collections/" + myCollection + "/leaders/" + myShardId + "/leader";
- }
-
- void setAmILeader(boolean amILeader) {
- if (this.amILeader != amILeader) {
- this.amILeader = amILeader;
- this.callback(); // notify the observers of a state change
- }
- }
-
- boolean amILeader() {
- return amILeader;
- }
-
- void shutdown() {
- if (wrappedWatcher != null) {
- wrappedWatcher.cancel(); // cancel the watcher to avoid spurious warn messages during shutdown
- }
- }
-
- private class LeaderStateWatcher implements Watcher {
-
- private boolean isCancelled = false;
-
- /**
- * Cancel the watcher to avoid spurious warn messages during shutdown.
- */
- void cancel() {
- isCancelled = true;
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (isCancelled) return; // if the watcher is cancelled, do nothing.
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-
- log.debug("The leader state has changed: {} @ {}:{}", event, collectionName, shard);
- // session events are not change events, and do not remove the watcher
- if (Event.EventType.None.equals(event.getType())) {
- return;
- }
-
- try {
- log.info("Received new leader state @ {}:{}", collectionName, shard);
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- ClusterState clusterState = core.getCoreContainer().getZkController().getClusterState();
- if (CdcrLeaderStateManager.this.isLeaderRegistered(zkClient, clusterState)) {
- CdcrLeaderStateManager.this.checkIfIAmLeader();
- }
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed updating leader state and setting watch @ {}: {}", collectionName, shard, e);
- }
- }
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrParams.java b/solr/core/src/java/org/apache/solr/handler/CdcrParams.java
deleted file mode 100644
index 3f65b90..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrParams.java
+++ /dev/null
@@ -1,256 +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 java.nio.charset.Charset;
-import java.util.Locale;
-
-public class CdcrParams {
-
- /**
- * The definition of a replica configuration *
- */
- public static final String REPLICA_PARAM = "replica";
-
- /**
- * The source collection of a replica *
- */
- public static final String SOURCE_COLLECTION_PARAM = "source";
-
- /**
- * The target collection of a replica *
- */
- public static final String TARGET_COLLECTION_PARAM = "target";
-
- /**
- * The Zookeeper host of the target cluster hosting the replica *
- */
- public static final String ZK_HOST_PARAM = "zkHost";
-
- /**
- * The definition of the {@link org.apache.solr.handler.CdcrReplicatorScheduler} configuration *
- */
- public static final String REPLICATOR_PARAM = "replicator";
-
- /**
- * The thread pool size of the replicator *
- */
- public static final String THREAD_POOL_SIZE_PARAM = "threadPoolSize";
-
- /**
- * The time schedule (in ms) of the replicator *
- */
- public static final String SCHEDULE_PARAM = "schedule";
-
- /**
- * The batch size of the replicator *
- */
- public static final String BATCH_SIZE_PARAM = "batchSize";
-
- /**
- * The definition of the {@link org.apache.solr.handler.CdcrUpdateLogSynchronizer} configuration *
- */
- public static final String UPDATE_LOG_SYNCHRONIZER_PARAM = "updateLogSynchronizer";
-
- /**
- * The definition of the {@link org.apache.solr.handler.CdcrBufferManager} configuration *
- */
- public static final String BUFFER_PARAM = "buffer";
-
- /**
- * The default state at startup of the buffer *
- */
- public static final String DEFAULT_STATE_PARAM = "defaultState";
-
- /**
- * The latest update checkpoint on a target cluster *
- */
- public final static String CHECKPOINT = "checkpoint";
-
- /**
- * The last processed version on a source cluster *
- */
- public final static String LAST_PROCESSED_VERSION = "lastProcessedVersion";
-
- /**
- * A list of replica queues on a source cluster *
- */
- public final static String QUEUES = "queues";
-
- /**
- * The size of a replica queue on a source cluster *
- */
- public final static String QUEUE_SIZE = "queueSize";
-
- /**
- * The timestamp of the last processed operation in a replica queue *
- */
- public final static String LAST_TIMESTAMP = "lastTimestamp";
-
- /**
- * A list of qps statistics per collection *
- */
- public final static String OPERATIONS_PER_SECOND = "operationsPerSecond";
-
- /**
- * Overall counter *
- */
- public final static String COUNTER_ALL = "all";
-
- /**
- * Counter for Adds *
- */
- public final static String COUNTER_ADDS = "adds";
-
- /**
- * Counter for Deletes *
- */
- public final static String COUNTER_DELETES = "deletes";
-
- /**
- * Counter for Bootstrap operations *
- */
- public final static String COUNTER_BOOTSTRAP = "bootstraps";
-
- /**
- * A list of errors per target collection *
- */
- public final static String ERRORS = "errors";
-
- /**
- * Counter for consecutive errors encountered by a replicator thread *
- */
- public final static String CONSECUTIVE_ERRORS = "consecutiveErrors";
-
- /**
- * A list of the last errors encountered by a replicator thread *
- */
- public final static String LAST = "last";
-
- /**
- * Total size of transaction logs *
- */
- public final static String TLOG_TOTAL_SIZE = "tlogTotalSize";
-
- /**
- * Total count of transaction logs *
- */
- public final static String TLOG_TOTAL_COUNT = "tlogTotalCount";
-
- /**
- * The state of the update log synchronizer *
- */
- public final static String UPDATE_LOG_SYNCHRONIZER = "updateLogSynchronizer";
-
- /**
- * The actions supported by the CDCR API
- */
- public enum CdcrAction {
- START,
- STOP,
- STATUS,
- COLLECTIONCHECKPOINT,
- SHARDCHECKPOINT,
- ENABLEBUFFER,
- DISABLEBUFFER,
- LASTPROCESSEDVERSION,
- QUEUES,
- OPS,
- ERRORS,
- BOOTSTRAP,
- BOOTSTRAP_STATUS,
- CANCEL_BOOTSTRAP;
-
- public static CdcrAction get(String p) {
- if (p != null) {
- try {
- return CdcrAction.valueOf(p.toUpperCase(Locale.ROOT));
- } catch (Exception e) {
- }
- }
- return null;
- }
-
- public String toLower() {
- return toString().toLowerCase(Locale.ROOT);
- }
-
- }
-
- /**
- * The possible states of the CDCR process
- */
- public enum ProcessState {
- STARTED,
- STOPPED;
-
- public static ProcessState get(byte[] state) {
- if (state != null) {
- try {
- return ProcessState.valueOf(new String(state, Charset.forName("UTF-8")).toUpperCase(Locale.ROOT));
- } catch (Exception e) {
- }
- }
- return null;
- }
-
- public String toLower() {
- return toString().toLowerCase(Locale.ROOT);
- }
-
- public byte[] getBytes() {
- return toLower().getBytes(Charset.forName("UTF-8"));
- }
-
- public static String getParam() {
- return "process";
- }
-
- }
-
- /**
- * The possible states of the CDCR buffer
- */
- public enum BufferState {
- ENABLED,
- DISABLED;
-
- public static BufferState get(byte[] state) {
- if (state != null) {
- try {
- return BufferState.valueOf(new String(state, Charset.forName("UTF-8")).toUpperCase(Locale.ROOT));
- } catch (Exception e) {
- }
- }
- return null;
- }
-
- public String toLower() {
- return toString().toLowerCase(Locale.ROOT);
- }
-
- public byte[] getBytes() {
- return toLower().getBytes(Charset.forName("UTF-8"));
- }
-
- public static String getParam() {
- return "buffer";
- }
-
- }
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
deleted file mode 100644
index 6506030..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
+++ /dev/null
@@ -1,178 +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 java.lang.invoke.MethodHandles;
-
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.core.SolrCore;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * Manage the life-cycle state of the CDCR process. It is responsible of synchronising the state
- * through Zookeeper. The state of the CDCR process is stored in the zk node defined by {@link #getZnodePath()}.
- * </p>
- * <p>
- * It takes care of notifying the {@link CdcrReplicatorManager} in case
- * of a process state change.
- * </p>
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-class CdcrProcessStateManager extends CdcrStateManager {
-
- private CdcrParams.ProcessState state = DEFAULT_STATE;
-
- private ProcessStateWatcher wrappedWatcher;
- private Watcher watcher;
-
- private SolrCore core;
-
- /**
- * The default state must be STOPPED. See comments in
- * {@link #setState(org.apache.solr.handler.CdcrParams.ProcessState)}.
- */
- static CdcrParams.ProcessState DEFAULT_STATE = CdcrParams.ProcessState.STOPPED;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrProcessStateManager(final SolrCore core) {
- this.core = core;
-
- // Ensure that the status znode exists
- this.createStateNode();
-
- // Register the watcher at startup
- try {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- watcher = this.initWatcher(zkClient);
- this.setState(CdcrParams.ProcessState.get(zkClient.getData(this.getZnodePath(), watcher, null, true)));
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed fetching initial state", e);
- }
- }
-
- /**
- * SolrZkClient does not guarantee that a watch object will only be triggered once for a given notification
- * if we does not wrap the watcher - see SOLR-6621.
- */
- private Watcher initWatcher(SolrZkClient zkClient) {
- wrappedWatcher = new ProcessStateWatcher();
- return zkClient.wrapWatcher(wrappedWatcher);
- }
-
- private String getZnodeBase() {
- return "/collections/" + core.getCoreDescriptor().getCloudDescriptor().getCollectionName() + "/cdcr/state";
- }
-
- private String getZnodePath() {
- return getZnodeBase() + "/process";
- }
-
- void setState(CdcrParams.ProcessState state) {
- if (this.state != state) {
- this.state = state;
- this.callback(); // notify the observers of a state change
- }
- }
-
- CdcrParams.ProcessState getState() {
- return state;
- }
-
- /**
- * Synchronise the state to Zookeeper. This method must be called only by the handler receiving the
- * action.
- */
- void synchronize() {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- zkClient.setData(this.getZnodePath(), this.getState().getBytes(), true);
- // check if nobody changed it in the meantime, and set a new watcher
- this.setState(CdcrParams.ProcessState.get(zkClient.getData(this.getZnodePath(), watcher, null, true)));
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed synchronising new state", e);
- }
- }
-
- private void createStateNode() {
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- if (!zkClient.exists(this.getZnodePath(), true)) {
- if (!zkClient.exists(this.getZnodeBase(), true)) { // Should be a no-op if the node exists
- zkClient.makePath(this.getZnodeBase(), null, CreateMode.PERSISTENT, null, false, true);
- }
- zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
- if (log.isInfoEnabled()) {
- log.info("Created znode {}", this.getZnodePath());
- }
- }
- } catch (KeeperException.NodeExistsException ne) {
- // Someone got in first and created the node.
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed to create CDCR process state node", e);
- }
- }
-
- void shutdown() {
- if (wrappedWatcher != null) {
- wrappedWatcher.cancel(); // cancel the watcher to avoid spurious warn messages during shutdown
- }
- }
-
- private class ProcessStateWatcher implements Watcher {
-
- private boolean isCancelled = false;
-
- /**
- * Cancel the watcher to avoid spurious warn messages during shutdown.
- */
- void cancel() {
- isCancelled = true;
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (isCancelled) return; // if the watcher is cancelled, do nothing.
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-
- log.info("The CDCR process state has changed: {} @ {}:{}", event, collectionName, shard);
- // session events are not change events, and do not remove the watcher
- if (Event.EventType.None.equals(event.getType())) {
- return;
- }
- SolrZkClient zkClient = core.getCoreContainer().getZkController().getZkClient();
- try {
- CdcrParams.ProcessState state = CdcrParams.ProcessState.get(zkClient.getData(CdcrProcessStateManager.this.getZnodePath(), watcher, null, true));
- log.info("Received new CDCR process state from watcher: {} @ {}:{}", state, collectionName, shard);
- CdcrProcessStateManager.this.setState(state);
- } catch (KeeperException | InterruptedException e) {
- log.warn("Failed synchronising new state @ {}: {}", collectionName, shard, e);
- }
- }
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java
deleted file mode 100644
index 936750e..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.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.handler;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.UpdateResponse;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.update.CdcrUpdateLog;
-import org.apache.solr.update.UpdateLog;
-import org.apache.solr.update.processor.CdcrUpdateProcessor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
-
-/**
- * The replication logic. Given a {@link org.apache.solr.handler.CdcrReplicatorState}, it reads all the new entries
- * in the update log and forward them to the target cluster. If an error occurs, the replication is stopped and
- * will be tried again later.
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-public class CdcrReplicator implements Runnable {
-
- private final CdcrReplicatorState state;
- private final int batchSize;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public CdcrReplicator(CdcrReplicatorState state, int batchSize) {
- this.state = state;
- this.batchSize = batchSize;
- }
-
- @Override
- public void run() {
- CdcrUpdateLog.CdcrLogReader logReader = state.getLogReader();
- CdcrUpdateLog.CdcrLogReader subReader = null;
- if (logReader == null) {
- log.warn("Log reader for target {} is not initialised, it will be ignored.", state.getTargetCollection());
- return;
- }
-
- try {
- // create update request
- UpdateRequest req = new UpdateRequest();
- // Add the param to indicate the {@link CdcrUpdateProcessor} to keep the provided version number
- req.setParam(CdcrUpdateProcessor.CDCR_UPDATE, "");
-
- // Start the benchmark timer
- state.getBenchmarkTimer().start();
-
- long counter = 0;
- subReader = logReader.getSubReader();
-
- for (int i = 0; i < batchSize; i++) {
- Object o = subReader.next();
- if (o == null) break; // we have reached the end of the update logs, we should close the batch
-
- if (isTargetCluster(o)) {
- continue;
- }
-
- if (isDelete(o)) {
-
- /*
- * Deletes are sent one at a time.
- */
-
- // First send out current batch of SolrInputDocument, the non-deletes.
- List<SolrInputDocument> docs = req.getDocuments();
-
- if (docs != null && docs.size() > 0) {
- subReader.resetToLastPosition(); // Push back the delete for now.
- this.sendRequest(req); // Send the batch update request
- logReader.forwardSeek(subReader); // Advance the main reader to just before the delete.
- o = subReader.next(); // Read the delete again
- counter += docs.size();
- req.clear();
- }
-
- // Process Delete
- this.processUpdate(o, req);
- this.sendRequest(req);
- logReader.forwardSeek(subReader);
- counter++;
- req.clear();
-
- } else {
-
- this.processUpdate(o, req);
-
- }
- }
-
- //Send the final batch out.
- List<SolrInputDocument> docs = req.getDocuments();
-
- if ((docs != null && docs.size() > 0)) {
- this.sendRequest(req);
- counter += docs.size();
- }
-
- // we might have read a single commit operation and reached the end of the update logs
- logReader.forwardSeek(subReader);
-
- if (log.isInfoEnabled()) {
- log.info("Forwarded {} updates to target {}", counter, state.getTargetCollection());
- }
- } catch (Exception e) {
- // report error and update error stats
- this.handleException(e);
- } finally {
- // stop the benchmark timer
- state.getBenchmarkTimer().stop();
- // ensure that the subreader is closed and the associated pointer is removed
- if (subReader != null) subReader.close();
- }
- }
-
- private void sendRequest(UpdateRequest req) throws IOException, SolrServerException, CdcrReplicatorException {
- UpdateResponse rsp = req.process(state.getClient());
- if (rsp.getStatus() != 0) {
- throw new CdcrReplicatorException(req, rsp);
- }
- state.resetConsecutiveErrors();
- }
-
- /** check whether the update read from TLog is received from source
- * or received via solr client
- */
- private boolean isTargetCluster(Object o) {
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- int operationAndFlags = (Integer) entry.get(0);
- int oper = operationAndFlags & UpdateLog.OPERATION_MASK;
- Boolean isTarget = false;
- if (oper == UpdateLog.DELETE_BY_QUERY || oper == UpdateLog.DELETE) {
- if (entry.size() == 4) { //back-combat - skip for previous versions
- isTarget = (Boolean) entry.get(entry.size() - 1);
- }
- } else if (oper == UpdateLog.UPDATE_INPLACE) {
- if (entry.size() == 6) { //back-combat - skip for previous versions
- isTarget = (Boolean) entry.get(entry.size() - 2);
- }
- } else if (oper == UpdateLog.ADD) {
- if (entry.size() == 4) { //back-combat - skip for previous versions
- isTarget = (Boolean) entry.get(entry.size() - 2);
- }
- }
- return isTarget;
- }
-
- private boolean isDelete(Object o) {
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- int operationAndFlags = (Integer) entry.get(0);
- int oper = operationAndFlags & UpdateLog.OPERATION_MASK;
- return oper == UpdateLog.DELETE_BY_QUERY || oper == UpdateLog.DELETE;
- }
-
- private void handleException(Exception e) {
- if (e instanceof CdcrReplicatorException) {
- UpdateRequest req = ((CdcrReplicatorException) e).req;
- UpdateResponse rsp = ((CdcrReplicatorException) e).rsp;
- log.warn("Failed to forward update request {} to target: {}. Got response {}", req, state.getTargetCollection(), rsp);
- state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
- } else if (e instanceof CloudSolrClient.RouteException) {
- log.warn("Failed to forward update request to target: {}", state.getTargetCollection(), e);
- state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
- } else {
- log.warn("Failed to forward update request to target: {}", state.getTargetCollection(), e);
- state.reportError(CdcrReplicatorState.ErrorType.INTERNAL);
- }
- }
-
- private UpdateRequest processUpdate(Object o, UpdateRequest req) {
-
- // should currently be a List<Oper,Ver,Doc/Id>
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
-
- int operationAndFlags = (Integer) entry.get(0);
- int oper = operationAndFlags & UpdateLog.OPERATION_MASK;
- long version = (Long) entry.get(1);
-
- // record the operation in the benchmark timer
- state.getBenchmarkTimer().incrementCounter(oper);
-
- switch (oper) {
- case UpdateLog.ADD: {
- // the version is already attached to the document
- SolrInputDocument sdoc = (SolrInputDocument) entry.get(entry.size() - 1);
- req.add(sdoc);
- return req;
- }
- case UpdateLog.DELETE: {
- byte[] idBytes = (byte[]) entry.get(2);
- req.deleteById(new String(idBytes, Charset.forName("UTF-8")));
- req.setParam(VERSION_FIELD, Long.toString(version));
- return req;
- }
-
- case UpdateLog.DELETE_BY_QUERY: {
- String query = (String) entry.get(2);
- req.deleteByQuery(query);
- req.setParam(VERSION_FIELD, Long.toString(version));
- return req;
- }
-
- case UpdateLog.COMMIT: {
- return null;
- }
-
- default:
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Operation! " + oper);
- }
- }
-
- /**
- * Exception to catch update request issues with the target cluster.
- */
- public static class CdcrReplicatorException extends Exception {
-
- private final UpdateRequest req;
- private final UpdateResponse rsp;
-
- public CdcrReplicatorException(UpdateRequest req, UpdateResponse rsp) {
- this.req = req;
- this.rsp = rsp;
- }
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
deleted file mode 100644
index 760e527..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
+++ /dev/null
@@ -1,441 +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 java.io.Closeable;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.http.client.HttpClient;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.update.CdcrUpdateLog;
-import org.apache.solr.util.TimeOut;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.handler.admin.CoreAdminHandler.RESPONSE_STATUS;
-
-@Deprecated(since = "8.6")
-class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
-
- private static final int MAX_BOOTSTRAP_ATTEMPTS = 5;
- private static final int BOOTSTRAP_RETRY_DELAY_MS = 2000;
- // 6 hours is hopefully long enough for most indexes
- private static final long BOOTSTRAP_TIMEOUT_SECONDS = 6L * 3600L * 3600L;
-
- private List<CdcrReplicatorState> replicatorStates;
-
- private final CdcrReplicatorScheduler scheduler;
- private CdcrProcessStateManager processStateManager;
- private CdcrLeaderStateManager leaderStateManager;
-
- private SolrCore core;
- private String path;
-
- private ExecutorService bootstrapExecutor;
- private volatile BootstrapStatusRunnable bootstrapStatusRunnable;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrReplicatorManager(final SolrCore core, String path,
- SolrParams replicatorConfiguration,
- Map<String, List<SolrParams>> replicasConfiguration) {
- this.core = core;
- this.path = path;
-
- // create states
- replicatorStates = new ArrayList<>();
- String myCollection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- List<SolrParams> targets = replicasConfiguration.get(myCollection);
- if (targets != null) {
- for (SolrParams params : targets) {
- String zkHost = params.get(CdcrParams.ZK_HOST_PARAM);
- String targetCollection = params.get(CdcrParams.TARGET_COLLECTION_PARAM);
-
- CloudSolrClient client = new Builder(Collections.singletonList(zkHost), Optional.empty())
- .withSocketTimeout(30000).withConnectionTimeout(15000)
- .sendUpdatesOnlyToShardLeaders()
- .build();
- client.setDefaultCollection(targetCollection);
- replicatorStates.add(new CdcrReplicatorState(targetCollection, zkHost, client));
- }
- }
-
- this.scheduler = new CdcrReplicatorScheduler(this, replicatorConfiguration);
- }
-
- void setProcessStateManager(final CdcrProcessStateManager processStateManager) {
- this.processStateManager = processStateManager;
- this.processStateManager.register(this);
- }
-
- void setLeaderStateManager(final CdcrLeaderStateManager leaderStateManager) {
- this.leaderStateManager = leaderStateManager;
- this.leaderStateManager.register(this);
- }
-
- /**
- * <p>
- * Inform the replicator manager of a change of state, and tell him to update its own state.
- * </p>
- * <p>
- * If we are the leader and the process state is STARTED, we need to initialise the log readers and start the
- * scheduled thread poll.
- * Otherwise, if the process state is STOPPED or if we are not the leader, we need to close the log readers and stop
- * the thread pool.
- * </p>
- * <p>
- * This method is synchronised as it can both be called by the leaderStateManager and the processStateManager.
- * </p>
- */
- @Override
- public synchronized void stateUpdate() {
- if (leaderStateManager.amILeader() && processStateManager.getState().equals(CdcrParams.ProcessState.STARTED)) {
- if (replicatorStates.size() > 0) {
- this.bootstrapExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(replicatorStates.size(),
- new SolrNamedThreadFactory("cdcr-bootstrap-status"));
- }
- this.initLogReaders();
- this.scheduler.start();
- return;
- }
-
- this.scheduler.shutdown();
- if (bootstrapExecutor != null) {
- IOUtils.closeQuietly(bootstrapStatusRunnable);
- ExecutorUtil.shutdownAndAwaitTermination(bootstrapExecutor);
- }
- this.closeLogReaders();
- @SuppressWarnings({"rawtypes"})
- Callable callable = core.getSolrCoreState().getCdcrBootstrapCallable();
- if (callable != null) {
- CdcrRequestHandler.BootstrapCallable bootstrapCallable = (CdcrRequestHandler.BootstrapCallable) callable;
- IOUtils.closeQuietly(bootstrapCallable);
- }
- }
-
- List<CdcrReplicatorState> getReplicatorStates() {
- return replicatorStates;
- }
-
- private void initLogReaders() {
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
- CdcrUpdateLog ulog = (CdcrUpdateLog) core.getUpdateHandler().getUpdateLog();
-
- for (CdcrReplicatorState state : replicatorStates) {
- state.closeLogReader();
- try {
- long checkpoint = this.getCheckpoint(state);
- if (log.isInfoEnabled()) {
- log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
- checkpoint, collectionName, shard);
- }
- CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
- boolean seek = reader.seek(checkpoint);
- state.init(reader);
- if (!seek) {
- // targetVersion is lower than the oldest known entry.
- // In this scenario, it probably means that there is a gap in the updates log.
- // the best we can do here is to bootstrap the target leader by replicating the full index
- final String targetCollection = state.getTargetCollection();
- state.setBootstrapInProgress(true);
- log.info("Attempting to bootstrap target collection: {}, shard: {}", targetCollection, shard);
- bootstrapStatusRunnable = new BootstrapStatusRunnable(core, state);
- log.info("Submitting bootstrap task to executor");
- try {
- bootstrapExecutor.submit(bootstrapStatusRunnable);
- } catch (Exception e) {
- log.error("Unable to submit bootstrap call to executor", e);
- }
- }
- } catch (IOException | SolrServerException | SolrException e) {
- log.warn("Unable to instantiate the log reader for target collection {}", state.getTargetCollection(), e);
- } catch (InterruptedException e) {
- log.warn("Thread interrupted while instantiate the log reader for target collection {}", state.getTargetCollection(), e);
- Thread.currentThread().interrupt();
- }
- }
- }
-
- private long getCheckpoint(CdcrReplicatorState state) throws IOException, SolrServerException {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.COLLECTIONCHECKPOINT.toString());
-
- @SuppressWarnings({"rawtypes"})
- SolrRequest request = new QueryRequest(params);
- request.setPath(path);
-
- @SuppressWarnings({"rawtypes"})
- NamedList response = state.getClient().request(request);
- return (Long) response.get(CdcrParams.CHECKPOINT);
- }
-
- void closeLogReaders() {
- for (CdcrReplicatorState state : replicatorStates) {
- state.closeLogReader();
- }
- }
-
- /**
- * Shutdown all the {@link org.apache.solr.handler.CdcrReplicatorState} by closing their
- * {@link org.apache.solr.client.solrj.impl.CloudSolrClient} and
- * {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader}.
- */
- void shutdown() {
- this.scheduler.shutdown();
- if (bootstrapExecutor != null) {
- IOUtils.closeQuietly(bootstrapStatusRunnable);
- ExecutorUtil.shutdownAndAwaitTermination(bootstrapExecutor);
- }
- for (CdcrReplicatorState state : replicatorStates) {
- state.shutdown();
- }
- replicatorStates.clear();
- }
-
- private class BootstrapStatusRunnable implements Runnable, Closeable {
- private final CdcrReplicatorState state;
- private final String targetCollection;
- private final String shard;
- private final String collectionName;
- private final CdcrUpdateLog ulog;
- private final String myCoreUrl;
-
- private volatile boolean closed = false;
-
- BootstrapStatusRunnable(SolrCore core, CdcrReplicatorState state) {
- this.collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- this.shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
- this.ulog = (CdcrUpdateLog) core.getUpdateHandler().getUpdateLog();
- this.state = state;
- this.targetCollection = state.getTargetCollection();
- String baseUrl = core.getCoreContainer().getZkController().getBaseUrl();
- this.myCoreUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, core.getName());
- }
-
- @Override
- public void close() throws IOException {
- closed = true;
- try {
- Replica leader = state.getClient().getZkStateReader().getLeaderRetry(targetCollection, shard, 30000); // assume same shard exists on target
- String leaderCoreUrl = leader.getCoreUrl();
- HttpClient httpClient = state.getClient().getLbClient().getHttpClient();
- try (HttpSolrClient client = new HttpSolrClient.Builder(leaderCoreUrl).withHttpClient(httpClient).build()) {
- sendCdcrCommand(client, CdcrParams.CdcrAction.CANCEL_BOOTSTRAP);
- } catch (SolrServerException e) {
- log.error("Error sending cancel bootstrap message to target collection: {} shard: {} leader: {}",
- targetCollection, shard, leaderCoreUrl);
- }
- } catch (InterruptedException e) {
- log.error("Interrupted while closing BootstrapStatusRunnable", e);
- Thread.currentThread().interrupt();
- }
- }
-
- @Override
- public void run() {
- int retries = 1;
- boolean success = false;
- try {
- while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
- Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
- }
- TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
- while (!timeOut.hasTimedOut()) {
- if (closed) {
- log.warn("Cancelling waiting for bootstrap on target: {} shard: {} to complete", targetCollection, shard);
- state.setBootstrapInProgress(false);
- break;
- }
- BootstrapStatus status = getBoostrapStatus();
- if (status == BootstrapStatus.RUNNING) {
- try {
- log.info("CDCR bootstrap running for {} seconds, sleeping for {} ms",
- BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS), BOOTSTRAP_RETRY_DELAY_MS);
- timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- } else if (status == BootstrapStatus.COMPLETED) {
- log.info("CDCR bootstrap successful in {} seconds", BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
- long checkpoint = CdcrReplicatorManager.this.getCheckpoint(state);
- if (log.isInfoEnabled()) {
- log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
- checkpoint, collectionName, shard);
- }
- CdcrUpdateLog.CdcrLogReader reader1 = ulog.newLogReader();
- reader1.seek(checkpoint);
- success = true;
- break;
- } else if (status == BootstrapStatus.FAILED) {
- log.warn("CDCR bootstrap failed in {} seconds", BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
- // let's retry a fixed number of times before giving up
- if (retries >= MAX_BOOTSTRAP_ATTEMPTS) {
- log.error("Unable to bootstrap the target collection: {}, shard: {} even after {} retries", targetCollection, shard, retries);
- break;
- } else {
- log.info("Retry: {} - Attempting to bootstrap target collection: {} shard: {}", retries, targetCollection, shard);
- while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
- Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
- }
- timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
- retries++;
- }
- } else if (status == BootstrapStatus.NOTFOUND || status == BootstrapStatus.CANCELLED) {
- if (log.isInfoEnabled()) {
- log.info("CDCR bootstrap {} in {} seconds"
- , (status == BootstrapStatus.NOTFOUND ? "not found" : "cancelled")
- , BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
- }
- // the leader of the target shard may have changed and therefore there is no record of the
- // bootstrap process so we must retry the operation
- while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
- Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
- }
- retries = 1;
- timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
- } else if (status == BootstrapStatus.UNKNOWN || status == BootstrapStatus.SUBMITTED) {
- if (log.isInfoEnabled()) {
- log.info("CDCR bootstrap is {} {}", (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
- BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
- }
- // we were not able to query the status on the remote end
- // so just sleep for a bit and try again
- timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
- }
- }
- } catch (InterruptedException e) {
- log.info("Bootstrap thread interrupted");
- state.reportError(CdcrReplicatorState.ErrorType.INTERNAL);
- Thread.currentThread().interrupt();
- } catch (IOException | SolrServerException | SolrException e) {
- log.error("Unable to bootstrap the target collection {} shard: {}", targetCollection, shard, e);
- state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
- } finally {
- if (success) {
- log.info("Bootstrap successful, giving the go-ahead to replicator");
- state.setBootstrapInProgress(false);
- }
- }
- }
-
- private BootstrapStatus sendBootstrapCommand() throws InterruptedException {
- Replica leader = state.getClient().getZkStateReader().getLeaderRetry(targetCollection, shard, 30000); // assume same shard exists on target
- String leaderCoreUrl = leader.getCoreUrl();
- HttpClient httpClient = state.getClient().getLbClient().getHttpClient();
- try (HttpSolrClient client = new HttpSolrClient.Builder(leaderCoreUrl).withHttpClient(httpClient).build()) {
- log.info("Attempting to bootstrap target collection: {} shard: {} leader: {}", targetCollection, shard, leaderCoreUrl);
- try {
- @SuppressWarnings({"rawtypes"})
- NamedList response = sendCdcrCommand(client, CdcrParams.CdcrAction.BOOTSTRAP, ReplicationHandler.LEADER_URL, myCoreUrl);
- log.debug("CDCR Bootstrap response: {}", response);
- String status = response.get(RESPONSE_STATUS).toString();
- return BootstrapStatus.valueOf(status.toUpperCase(Locale.ROOT));
- } catch (Exception e) {
- log.error("Exception submitting bootstrap request", e);
- return BootstrapStatus.UNKNOWN;
- }
- } catch (IOException e) {
- log.error("There shouldn't be an IOException while closing but there was!", e);
- }
- return BootstrapStatus.UNKNOWN;
- }
-
- private BootstrapStatus getBoostrapStatus() throws InterruptedException {
- try {
- Replica leader = state.getClient().getZkStateReader().getLeaderRetry(targetCollection, shard, 30000); // assume same shard exists on target
- String leaderCoreUrl = leader.getCoreUrl();
- HttpClient httpClient = state.getClient().getLbClient().getHttpClient();
- try (HttpSolrClient client = new HttpSolrClient.Builder(leaderCoreUrl).withHttpClient(httpClient).build()) {
- @SuppressWarnings({"rawtypes"})
- NamedList response = sendCdcrCommand(client, CdcrParams.CdcrAction.BOOTSTRAP_STATUS);
- String status = (String) response.get(RESPONSE_STATUS);
- BootstrapStatus bootstrapStatus = BootstrapStatus.valueOf(status.toUpperCase(Locale.ROOT));
- if (bootstrapStatus == BootstrapStatus.RUNNING) {
- return BootstrapStatus.RUNNING;
- } else if (bootstrapStatus == BootstrapStatus.COMPLETED) {
- return BootstrapStatus.COMPLETED;
- } else if (bootstrapStatus == BootstrapStatus.FAILED) {
- return BootstrapStatus.FAILED;
- } else if (bootstrapStatus == BootstrapStatus.NOTFOUND) {
- log.warn("Bootstrap process was not found on target collection: {} shard: {}, leader: {}", targetCollection, shard, leaderCoreUrl);
- return BootstrapStatus.NOTFOUND;
- } else if (bootstrapStatus == BootstrapStatus.CANCELLED) {
- return BootstrapStatus.CANCELLED;
- } else {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unknown status: " + status + " returned by BOOTSTRAP_STATUS command");
- }
- }
- } catch (Exception e) {
- log.error("Exception during bootstrap status request", e);
- return BootstrapStatus.UNKNOWN;
- }
- }
- }
-
- @SuppressWarnings({"rawtypes"})
- private NamedList sendCdcrCommand(SolrClient client, CdcrParams.CdcrAction action, String... params) throws SolrServerException, IOException {
- ModifiableSolrParams solrParams = new ModifiableSolrParams();
- solrParams.set(CommonParams.QT, "/cdcr");
- solrParams.set(CommonParams.ACTION, action.toString());
- for (int i = 0; i < params.length - 1; i+=2) {
- solrParams.set(params[i], params[i + 1]);
- }
- SolrRequest request = new QueryRequest(solrParams);
- return client.request(request);
- }
-
- private enum BootstrapStatus {
- SUBMITTED,
- RUNNING,
- COMPLETED,
- FAILED,
- NOTFOUND,
- CANCELLED,
- UNKNOWN
- }
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorScheduler.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorScheduler.java
deleted file mode 100644
index 1418465..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorScheduler.java
+++ /dev/null
@@ -1,116 +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.common.params.SolrParams;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-import java.util.concurrent.*;
-
-/**
- * Schedule the execution of the {@link org.apache.solr.handler.CdcrReplicator} threads at
- * regular time interval. It relies on a queue of {@link org.apache.solr.handler.CdcrReplicatorState} in
- * order to avoid that one {@link org.apache.solr.handler.CdcrReplicatorState} is used by two threads at the same
- * time.
- */
-class CdcrReplicatorScheduler {
-
- private boolean isStarted = false;
-
- private ScheduledExecutorService scheduler;
- private ExecutorService replicatorsPool;
-
- private final CdcrReplicatorManager replicatorManager;
- private final ConcurrentLinkedQueue<CdcrReplicatorState> statesQueue;
-
- private int poolSize = DEFAULT_POOL_SIZE;
- private int timeSchedule = DEFAULT_TIME_SCHEDULE;
- private int batchSize = DEFAULT_BATCH_SIZE;
-
- private static final int DEFAULT_POOL_SIZE = 2;
- private static final int DEFAULT_TIME_SCHEDULE = 10;
- private static final int DEFAULT_BATCH_SIZE = 128;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrReplicatorScheduler(final CdcrReplicatorManager replicatorStatesManager, final SolrParams replicatorConfiguration) {
- this.replicatorManager = replicatorStatesManager;
- this.statesQueue = new ConcurrentLinkedQueue<>(replicatorManager.getReplicatorStates());
- if (replicatorConfiguration != null) {
- poolSize = replicatorConfiguration.getInt(CdcrParams.THREAD_POOL_SIZE_PARAM, DEFAULT_POOL_SIZE);
- timeSchedule = replicatorConfiguration.getInt(CdcrParams.SCHEDULE_PARAM, DEFAULT_TIME_SCHEDULE);
- batchSize = replicatorConfiguration.getInt(CdcrParams.BATCH_SIZE_PARAM, DEFAULT_BATCH_SIZE);
- }
- }
-
- void start() {
- if (!isStarted) {
- scheduler = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-scheduler"));
- replicatorsPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("cdcr-replicator"));
-
- // the scheduler thread is executed every second and submits one replication task
- // per available state in the queue
- scheduler.scheduleWithFixedDelay(() -> {
- int nCandidates = statesQueue.size();
- for (int i = 0; i < nCandidates; i++) {
- // a thread that poll one state from the queue, execute the replication task, and push back
- // the state in the queue when the task is completed
- replicatorsPool.execute(() -> {
- CdcrReplicatorState state = statesQueue.poll();
- assert state != null; // Should never happen
- try {
- if (!state.isBootstrapInProgress()) {
- new CdcrReplicator(state, batchSize).run();
- } else {
- if (log.isDebugEnabled()) {
- log.debug("Replicator state is bootstrapping, skipping replication for target collection {}", state.getTargetCollection());
- }
- }
- } finally {
- statesQueue.offer(state);
- }
- });
-
- }
- }, 0, timeSchedule, TimeUnit.MILLISECONDS);
- isStarted = true;
- }
- }
-
- void shutdown() {
- if (isStarted) {
- // interrupts are often dangerous in Lucene / Solr code, but the
- // test for this will leak threads without
- replicatorsPool.shutdown();
- try {
- replicatorsPool.awaitTermination(60, TimeUnit.SECONDS);
- } catch (InterruptedException e) {
- log.warn("Thread interrupted while waiting for CDCR replicator threadpool close.");
- Thread.currentThread().interrupt();
- } finally {
- scheduler.shutdownNow();
- isStarted = false;
- }
- }
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java
deleted file mode 100644
index af9020a..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java
+++ /dev/null
@@ -1,299 +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 java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.update.CdcrUpdateLog;
-import org.apache.solr.update.UpdateLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The state of the replication with a target cluster.
- */
-class CdcrReplicatorState {
-
- private final String targetCollection;
- private final String zkHost;
- private final CloudSolrClient targetClient;
-
- private CdcrUpdateLog.CdcrLogReader logReader;
-
- private long consecutiveErrors = 0;
- private final Map<ErrorType, Long> errorCounters = new HashMap<>();
- private final FixedQueue<ErrorQueueEntry> errorsQueue = new FixedQueue<>(100); // keep the last 100 errors
-
- private BenchmarkTimer benchmarkTimer;
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private final AtomicBoolean bootstrapInProgress = new AtomicBoolean(false);
- private final AtomicInteger numBootstraps = new AtomicInteger();
-
- CdcrReplicatorState(final String targetCollection, final String zkHost, final CloudSolrClient targetClient) {
- this.targetCollection = targetCollection;
- this.targetClient = targetClient;
- this.zkHost = zkHost;
- this.benchmarkTimer = new BenchmarkTimer();
- }
-
- /**
- * Initialise the replicator state with a {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader}
- * that is positioned at the last target cluster checkpoint.
- */
- void init(final CdcrUpdateLog.CdcrLogReader logReader) {
- this.logReader = logReader;
- }
-
- void closeLogReader() {
- if (logReader != null) {
- logReader.close();
- logReader = null;
- }
- }
-
- CdcrUpdateLog.CdcrLogReader getLogReader() {
- return logReader;
- }
-
- String getTargetCollection() {
- return targetCollection;
- }
-
- String getZkHost() {
- return zkHost;
- }
-
- CloudSolrClient getClient() {
- return targetClient;
- }
-
- void shutdown() {
- try {
- targetClient.close();
- } catch (IOException ioe) {
- log.warn("Caught exception trying to close server: ", ioe);
- }
- this.closeLogReader();
- }
-
- void reportError(ErrorType error) {
- if (!errorCounters.containsKey(error)) {
- errorCounters.put(error, 0l);
- }
- errorCounters.put(error, errorCounters.get(error) + 1);
- errorsQueue.add(new ErrorQueueEntry(error, new Date()));
- consecutiveErrors++;
- }
-
- void resetConsecutiveErrors() {
- consecutiveErrors = 0;
- }
-
- /**
- * Returns the number of consecutive errors encountered while trying to forward updates to the target.
- */
- long getConsecutiveErrors() {
- return consecutiveErrors;
- }
-
- /**
- * Gets the number of errors of a particular type.
- */
- long getErrorCount(ErrorType type) {
- if (errorCounters.containsKey(type)) {
- return errorCounters.get(type);
- } else {
- return 0;
- }
- }
-
- /**
- * Gets the last errors ordered by timestamp (most recent first)
- */
- List<String[]> getLastErrors() {
- List<String[]> lastErrors = new ArrayList<>();
- synchronized (errorsQueue) {
- Iterator<ErrorQueueEntry> it = errorsQueue.iterator();
- while (it.hasNext()) {
- ErrorQueueEntry entry = it.next();
- lastErrors.add(new String[]{entry.timestamp.toInstant().toString(), entry.type.toLower()});
- }
- }
- return lastErrors;
- }
-
- /**
- * Return the timestamp of the last processed operations
- */
- String getTimestampOfLastProcessedOperation() {
- if (logReader != null && logReader.getLastVersion() != -1) {
- // Shift back to the right by 20 bits the version number - See VersionInfo#getNewClock
- return Instant.ofEpochMilli(logReader.getLastVersion() >> 20).toString();
- }
- return "";
- }
-
- /**
- * Gets the benchmark timer.
- */
- BenchmarkTimer getBenchmarkTimer() {
- return this.benchmarkTimer;
- }
-
- /**
- * @return true if a bootstrap operation is in progress, false otherwise
- */
- boolean isBootstrapInProgress() {
- return bootstrapInProgress.get();
- }
-
- void setBootstrapInProgress(boolean inProgress) {
- if (bootstrapInProgress.compareAndSet(true, false)) {
- numBootstraps.incrementAndGet();
- }
- bootstrapInProgress.set(inProgress);
- }
-
- public int getNumBootstraps() {
- return numBootstraps.get();
- }
-
- enum ErrorType {
- INTERNAL,
- BAD_REQUEST;
-
- public String toLower() {
- return toString().toLowerCase(Locale.ROOT);
- }
-
- }
-
- static class BenchmarkTimer {
-
- private long startTime;
- private long runTime = 0;
- private Map<Integer, Long> opCounters = new HashMap<>();
-
- /**
- * Start recording time.
- */
- void start() {
- startTime = System.nanoTime();
- }
-
- /**
- * Stop recording time.
- */
- void stop() {
- runTime += System.nanoTime() - startTime;
- startTime = -1;
- }
-
- void incrementCounter(final int operationType) {
- switch (operationType) {
- case UpdateLog.ADD:
- case UpdateLog.DELETE:
- case UpdateLog.DELETE_BY_QUERY: {
- if (!opCounters.containsKey(operationType)) {
- opCounters.put(operationType, 0l);
- }
- opCounters.put(operationType, opCounters.get(operationType) + 1);
- return;
- }
-
- default:
- }
- }
-
- long getRunTime() {
- long totalRunTime = runTime;
- if (startTime != -1) { // we are currently recording the time
- totalRunTime += System.nanoTime() - startTime;
- }
- return totalRunTime;
- }
-
- double getOperationsPerSecond() {
- long total = 0;
- for (long counter : opCounters.values()) {
- total += counter;
- }
- double elapsedTimeInSeconds = ((double) this.getRunTime() / 1E9);
- return total / elapsedTimeInSeconds;
- }
-
- double getAddsPerSecond() {
- long total = opCounters.get(UpdateLog.ADD) != null ? opCounters.get(UpdateLog.ADD) : 0;
- double elapsedTimeInSeconds = ((double) this.getRunTime() / 1E9);
- return total / elapsedTimeInSeconds;
- }
-
- double getDeletesPerSecond() {
- long total = opCounters.get(UpdateLog.DELETE) != null ? opCounters.get(UpdateLog.DELETE) : 0;
- total += opCounters.get(UpdateLog.DELETE_BY_QUERY) != null ? opCounters.get(UpdateLog.DELETE_BY_QUERY) : 0;
- double elapsedTimeInSeconds = ((double) this.getRunTime() / 1E9);
- return total / elapsedTimeInSeconds;
- }
-
- }
-
- private static class ErrorQueueEntry {
-
- private ErrorType type;
- private Date timestamp;
-
- private ErrorQueueEntry(ErrorType type, Date timestamp) {
- this.type = type;
- this.timestamp = timestamp;
- }
- }
-
- private static class FixedQueue<E> extends LinkedList<E> {
-
- private int maxSize;
-
- public FixedQueue(int maxSize) {
- this.maxSize = maxSize;
- }
-
- @Override
- public synchronized boolean add(E e) {
- super.addFirst(e);
- if (size() > maxSize) {
- removeLast();
- }
- return true;
- }
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
deleted file mode 100644
index a9dfeae..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
+++ /dev/null
@@ -1,880 +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 java.io.Closeable;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.stream.Collectors;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cloud.ZkShardTerms;
-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.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.params.UpdateParams;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.core.CloseHook;
-import org.apache.solr.core.PluginBag;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.update.CdcrUpdateLog;
-import org.apache.solr.update.SolrCoreState;
-import org.apache.solr.update.UpdateLog;
-import org.apache.solr.update.VersionInfo;
-import org.apache.solr.update.processor.DistributedUpdateProcessor;
-import org.apache.solr.util.plugin.SolrCoreAware;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.handler.admin.CoreAdminHandler.COMPLETED;
-import static org.apache.solr.handler.admin.CoreAdminHandler.FAILED;
-import static org.apache.solr.handler.admin.CoreAdminHandler.RESPONSE;
-import static org.apache.solr.handler.admin.CoreAdminHandler.RESPONSE_MESSAGE;
-import static org.apache.solr.handler.admin.CoreAdminHandler.RESPONSE_STATUS;
-import static org.apache.solr.handler.admin.CoreAdminHandler.RUNNING;
-
-/**
- * <p>
- * This request handler implements the CDCR API and is responsible of the execution of the
- * {@link CdcrReplicator} threads.
- * </p>
- * <p>
- * It relies on three classes, {@link org.apache.solr.handler.CdcrLeaderStateManager},
- * {@link org.apache.solr.handler.CdcrBufferStateManager} and {@link org.apache.solr.handler.CdcrProcessStateManager}
- * to synchronise the state of the CDCR across all the nodes.
- * </p>
- * <p>
- * The CDCR process can be either {@link org.apache.solr.handler.CdcrParams.ProcessState#STOPPED} or {@link org.apache.solr.handler.CdcrParams.ProcessState#STARTED} by using the
- * actions {@link org.apache.solr.handler.CdcrParams.CdcrAction#STOP} and {@link org.apache.solr.handler.CdcrParams.CdcrAction#START} respectively. If a node is leader and the process
- * state is {@link org.apache.solr.handler.CdcrParams.ProcessState#STARTED}, the {@link CdcrReplicatorManager} will
- * start the {@link CdcrReplicator} threads. If a node becomes non-leader or if the process state becomes
- * {@link org.apache.solr.handler.CdcrParams.ProcessState#STOPPED}, the {@link CdcrReplicator} threads are stopped.
- * </p>
- * <p>
- * The CDCR can be switched to a "buffering" mode, in which the update log will never delete old transaction log
- * files. Such a mode can be enabled or disabled using the action {@link org.apache.solr.handler.CdcrParams.CdcrAction#ENABLEBUFFER} and
- * {@link org.apache.solr.handler.CdcrParams.CdcrAction#DISABLEBUFFER} respectively.
- * </p>
- * <p>
- * Known limitations: The source and target clusters must have the same topology. Replication between clusters
- * with a different number of shards will likely results in an inconsistent index.
- * </p>
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAware {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private SolrCore core;
- private String collection;
- private String shard;
- private String path;
-
- private SolrParams updateLogSynchronizerConfiguration;
- private SolrParams replicatorConfiguration;
- private SolrParams bufferConfiguration;
- private Map<String, List<SolrParams>> replicasConfiguration;
-
- private CdcrProcessStateManager processStateManager;
- private CdcrBufferStateManager bufferStateManager;
- private CdcrReplicatorManager replicatorManager;
- private CdcrLeaderStateManager leaderStateManager;
- private CdcrUpdateLogSynchronizer updateLogSynchronizer;
- private CdcrBufferManager bufferManager;
-
- @Override
- public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
- super.init(args);
-
- log.warn("CDCR (in its current form) is deprecated as of 8.6 and shall be removed in 9.0. See SOLR-14022 for details.");
-
- if (args != null) {
- // Configuration of the Update Log Synchronizer
- Object updateLogSynchonizerParam = args.get(CdcrParams.UPDATE_LOG_SYNCHRONIZER_PARAM);
- if (updateLogSynchonizerParam != null && updateLogSynchonizerParam instanceof NamedList) {
- updateLogSynchronizerConfiguration = ((NamedList) updateLogSynchonizerParam).toSolrParams();
- }
-
- // Configuration of the Replicator
- Object replicatorParam = args.get(CdcrParams.REPLICATOR_PARAM);
- if (replicatorParam != null && replicatorParam instanceof NamedList) {
- replicatorConfiguration = ((NamedList) replicatorParam).toSolrParams();
- }
-
- // Configuration of the Buffer
- Object bufferParam = args.get(CdcrParams.BUFFER_PARAM);
- if (bufferParam != null && bufferParam instanceof NamedList) {
- bufferConfiguration = ((NamedList) bufferParam).toSolrParams();
- }
-
- // Configuration of the Replicas
- replicasConfiguration = new HashMap<>();
- @SuppressWarnings({"rawtypes"})
- List replicas = args.getAll(CdcrParams.REPLICA_PARAM);
- for (Object replica : replicas) {
- if (replica != null && replica instanceof NamedList) {
- SolrParams params = ((NamedList) replica).toSolrParams();
- if (!replicasConfiguration.containsKey(params.get(CdcrParams.SOURCE_COLLECTION_PARAM))) {
- replicasConfiguration.put(params.get(CdcrParams.SOURCE_COLLECTION_PARAM), new ArrayList<>());
- }
- replicasConfiguration.get(params.get(CdcrParams.SOURCE_COLLECTION_PARAM)).add(params);
- }
- }
- }
- }
-
- @Override
- public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
- // Pick the action
- SolrParams params = req.getParams();
- CdcrParams.CdcrAction action = null;
- String a = params.get(CommonParams.ACTION);
- if (a != null) {
- action = CdcrParams.CdcrAction.get(a);
- }
- if (action == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
- }
-
- switch (action) {
- case START: {
- this.handleStartAction(req, rsp);
- break;
- }
- case STOP: {
- this.handleStopAction(req, rsp);
- break;
- }
- case STATUS: {
- this.handleStatusAction(req, rsp);
- break;
- }
- case COLLECTIONCHECKPOINT: {
- this.handleCollectionCheckpointAction(req, rsp);
- break;
- }
- case SHARDCHECKPOINT: {
- this.handleShardCheckpointAction(req, rsp);
- break;
- }
- case ENABLEBUFFER: {
- this.handleEnableBufferAction(req, rsp);
- break;
- }
- case DISABLEBUFFER: {
- this.handleDisableBufferAction(req, rsp);
- break;
- }
- case LASTPROCESSEDVERSION: {
- this.handleLastProcessedVersionAction(req, rsp);
- break;
- }
- case QUEUES: {
- this.handleQueuesAction(req, rsp);
- break;
- }
- case OPS: {
- this.handleOpsAction(req, rsp);
- break;
- }
- case ERRORS: {
- this.handleErrorsAction(req, rsp);
- break;
- }
- case BOOTSTRAP: {
- this.handleBootstrapAction(req, rsp);
- break;
- }
- case BOOTSTRAP_STATUS: {
- this.handleBootstrapStatus(req, rsp);
- break;
- }
- case CANCEL_BOOTSTRAP: {
- this.handleCancelBootstrap(req, rsp);
- break;
- }
- default: {
- throw new RuntimeException("Unknown action: " + action);
- }
- }
-
- rsp.setHttpCaching(false);
- }
-
- @Override
- public void inform(SolrCore core) {
- this.core = core;
- collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-
- // Make sure that the core is ZKAware
- if (!core.getCoreContainer().isZooKeeperAware()) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Solr instance is not running in SolrCloud mode.");
- }
-
- // Make sure that the core is using the CdcrUpdateLog implementation
- if (!(core.getUpdateHandler().getUpdateLog() instanceof CdcrUpdateLog)) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Solr instance is not configured with the cdcr update log.");
- }
-
- // Find the registered path of the handler
- path = null;
- for (Map.Entry<String, PluginBag.PluginHolder<SolrRequestHandler>> entry : core.getRequestHandlers().getRegistry().entrySet()) {
- if (core.getRequestHandlers().isLoaded(entry.getKey()) && entry.getValue().get() == this) {
- path = entry.getKey();
- break;
- }
- }
- if (path == null) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "The CdcrRequestHandler is not registered with the current core.");
- }
- if (!path.startsWith("/")) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "The CdcrRequestHandler needs to be registered to a path. Typically this is '/cdcr'");
- }
-
- // Initialisation phase
- // If the Solr cloud is being initialised, each CDCR node will start up in its default state, i.e., STOPPED
- // and non-leader. The leader state will be updated later, when all the Solr cores have been loaded.
- // If the Solr cloud has already been initialised, and the core is reloaded (i.e., because a node died or a new node
- // is added to the cluster), the CDCR node will synchronise its state with the global CDCR state that is stored
- // in zookeeper.
-
- // Initialise the buffer state manager
- bufferStateManager = new CdcrBufferStateManager(core, bufferConfiguration);
- // Initialise the process state manager
- processStateManager = new CdcrProcessStateManager(core);
- // Initialise the leader state manager
- leaderStateManager = new CdcrLeaderStateManager(core);
-
- // Initialise the replicator states manager
- replicatorManager = new CdcrReplicatorManager(core, path, replicatorConfiguration, replicasConfiguration);
- replicatorManager.setProcessStateManager(processStateManager);
- replicatorManager.setLeaderStateManager(leaderStateManager);
- // we need to inform it of a state event since the process and leader state
- // may have been synchronised during the initialisation
- replicatorManager.stateUpdate();
-
- // Initialise the update log synchronizer
- updateLogSynchronizer = new CdcrUpdateLogSynchronizer(core, path, updateLogSynchronizerConfiguration);
- updateLogSynchronizer.setLeaderStateManager(leaderStateManager);
- // we need to inform it of a state event since the leader state
- // may have been synchronised during the initialisation
- updateLogSynchronizer.stateUpdate();
-
- // Initialise the buffer manager
- bufferManager = new CdcrBufferManager(core);
- bufferManager.setLeaderStateManager(leaderStateManager);
- bufferManager.setBufferStateManager(bufferStateManager);
- // we need to inform it of a state event since the leader state
- // may have been synchronised during the initialisation
- bufferManager.stateUpdate();
-
- // register the close hook
- this.registerCloseHook(core);
- }
-
- /**
- * register a close hook to properly shutdown the state manager and scheduler
- */
- private void registerCloseHook(SolrCore core) {
- core.addCloseHook(new CloseHook() {
-
- @Override
- public void preClose(SolrCore core) {
- log.info("Solr core is being closed - shutting down CDCR handler @ {}:{}", collection, shard);
-
- updateLogSynchronizer.shutdown();
- replicatorManager.shutdown();
- bufferStateManager.shutdown();
- processStateManager.shutdown();
- leaderStateManager.shutdown();
- }
-
- @Override
- public void postClose(SolrCore core) {
- }
-
- });
- }
-
- /**
- * <p>
- * Update and synchronize the process state.
- * </p>
- * <p>
- * The process state manager must notify the replicator states manager of the change of state.
- * </p>
- */
- private void handleStartAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- if (processStateManager.getState() == CdcrParams.ProcessState.STOPPED) {
- processStateManager.setState(CdcrParams.ProcessState.STARTED);
- processStateManager.synchronize();
- }
-
- rsp.add(CdcrParams.CdcrAction.STATUS.toLower(), this.getStatus());
- }
-
- private void handleStopAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- if (processStateManager.getState() == CdcrParams.ProcessState.STARTED) {
- processStateManager.setState(CdcrParams.ProcessState.STOPPED);
- processStateManager.synchronize();
- }
-
- rsp.add(CdcrParams.CdcrAction.STATUS.toLower(), this.getStatus());
- }
-
- private void handleStatusAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- rsp.add(CdcrParams.CdcrAction.STATUS.toLower(), this.getStatus());
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- private NamedList getStatus() {
- NamedList status = new NamedList();
- status.add(CdcrParams.ProcessState.getParam(), processStateManager.getState().toLower());
- status.add(CdcrParams.BufferState.getParam(), bufferStateManager.getState().toLower());
- return status;
- }
-
- /**
- * This action is generally executed on the target cluster in order to retrieve the latest update checkpoint.
- * This checkpoint is used on the source cluster to setup the
- * {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader} of a shard leader. <br/>
- * This method will execute in parallel one
- * {@link org.apache.solr.handler.CdcrParams.CdcrAction#SHARDCHECKPOINT} request per shard leader. It will
- * then pick the lowest version number as checkpoint. Picking the lowest amongst all shards will ensure that we do not
- * pick a checkpoint that is ahead of the source cluster. This can occur when other shard leaders are sending new
- * updates to the target cluster while we are currently instantiating the
- * {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader}.
- * This solution only works in scenarios where the topology of the source and target clusters are identical.
- */
- private void handleCollectionCheckpointAction(SolrQueryRequest req, SolrQueryResponse rsp)
- throws IOException, SolrServerException {
- ZkController zkController = core.getCoreContainer().getZkController();
- try {
- zkController.getZkStateReader().forceUpdateCollection(collection);
- } catch (Exception e) {
- log.warn("Error when updating cluster state", e);
- }
- ClusterState cstate = zkController.getClusterState();
- DocCollection docCollection = cstate.getCollectionOrNull(collection);
- Collection<Slice> shards = docCollection == null? null : docCollection.getActiveSlices();
-
- ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("parallelCdcrExecutor"));
-
- long checkpoint = Long.MAX_VALUE;
- try {
- List<Callable<Long>> callables = new ArrayList<>();
- for (Slice shard : shards) {
- ZkNodeProps leaderProps = zkController.getZkStateReader().getLeaderRetry(collection, shard.getName());
- ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
- callables.add(new SliceCheckpointCallable(nodeProps.getCoreUrl(), path));
- }
-
- for (final Future<Long> future : parallelExecutor.invokeAll(callables)) {
- long version = future.get();
- if (version < checkpoint) { // we must take the lowest checkpoint from all the shards
- checkpoint = version;
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Error while requesting shard's checkpoints", e);
- } catch (ExecutionException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Error while requesting shard's checkpoints", e);
- } finally {
- parallelExecutor.shutdown();
- }
-
- rsp.add(CdcrParams.CHECKPOINT, checkpoint);
- }
-
- /**
- * Retrieve the version number of the latest entry of the {@link org.apache.solr.update.UpdateLog}.
- */
- private void handleShardCheckpointAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- if (!leaderStateManager.amILeader()) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Action '" + CdcrParams.CdcrAction.SHARDCHECKPOINT +
- "' sent to non-leader replica");
- }
-
- UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
- VersionInfo versionInfo = ulog.getVersionInfo();
- try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
- long maxVersionFromRecent = recentUpdates.getMaxRecentVersion();
- long maxVersionFromIndex = versionInfo.getMaxVersionFromIndex(req.getSearcher());
- log.info("Found maxVersionFromRecent {} maxVersionFromIndex {}", maxVersionFromRecent, maxVersionFromIndex);
- // there is no race with ongoing bootstrap because we don't expect any updates to come from the source
- long maxVersion = Math.max(maxVersionFromIndex, maxVersionFromRecent);
- if (maxVersion == 0L) {
- maxVersion = -1;
- }
- rsp.add(CdcrParams.CHECKPOINT, maxVersion);
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Action '" + CdcrParams.CdcrAction.SHARDCHECKPOINT +
- "' could not read max version");
- }
- }
-
- private void handleEnableBufferAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- if (bufferStateManager.getState() == CdcrParams.BufferState.DISABLED) {
- bufferStateManager.setState(CdcrParams.BufferState.ENABLED);
- bufferStateManager.synchronize();
- }
-
- rsp.add(CdcrParams.CdcrAction.STATUS.toLower(), this.getStatus());
- }
-
- private void handleDisableBufferAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- if (bufferStateManager.getState() == CdcrParams.BufferState.ENABLED) {
- bufferStateManager.setState(CdcrParams.BufferState.DISABLED);
- bufferStateManager.synchronize();
- }
-
- rsp.add(CdcrParams.CdcrAction.STATUS.toLower(), this.getStatus());
- }
-
- /**
- * <p>
- * We have to take care of four cases:
- * <ul>
- * <li>Replication & Buffering</li>
- * <li>Replication & No Buffering</li>
- * <li>No Replication & Buffering</li>
- * <li>No Replication & No Buffering</li>
- * </ul>
- * In the first three cases, at least one log reader should have been initialised. We should take the lowest
- * last processed version across all the initialised readers. In the last case, there isn't a log reader
- * initialised. We should instantiate one and get the version of the first entries.
- * </p>
- */
- private void handleLastProcessedVersionAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-
- if (!leaderStateManager.amILeader()) {
- log.warn("Action {} sent to non-leader replica @ {}:{}", CdcrParams.CdcrAction.LASTPROCESSEDVERSION, collectionName, shard);
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Action " + CdcrParams.CdcrAction.LASTPROCESSEDVERSION +
- " sent to non-leader replica");
- }
-
- // take care of the first three cases
- // first check the log readers from the replicator states
- long lastProcessedVersion = Long.MAX_VALUE;
- for (CdcrReplicatorState state : replicatorManager.getReplicatorStates()) {
- long version = Long.MAX_VALUE;
- if (state.getLogReader() != null) {
- version = state.getLogReader().getLastVersion();
- }
- lastProcessedVersion = Math.min(lastProcessedVersion, version);
- }
-
- // next check the log reader of the buffer
- CdcrUpdateLog.CdcrLogReader bufferLogReader = ((CdcrUpdateLog) core.getUpdateHandler().getUpdateLog()).getBufferToggle();
- if (bufferLogReader != null) {
- lastProcessedVersion = Math.min(lastProcessedVersion, bufferLogReader.getLastVersion());
- }
-
- // the fourth case: no cdc replication, no buffering: all readers were null
- if (processStateManager.getState().equals(CdcrParams.ProcessState.STOPPED) &&
- bufferStateManager.getState().equals(CdcrParams.BufferState.DISABLED)) {
- CdcrUpdateLog.CdcrLogReader logReader = ((CdcrUpdateLog) core.getUpdateHandler().getUpdateLog()).newLogReader();
- try {
- // let the reader initialize lastVersion
- logReader.next();
- lastProcessedVersion = Math.min(lastProcessedVersion, logReader.getLastVersion());
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Error while fetching the last processed version", e);
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Error while fetching the last processed version", e);
- } finally {
- logReader.close();
- }
- }
-
- log.debug("Returning the lowest last processed version {} @ {}:{}", lastProcessedVersion, collectionName, shard);
- rsp.add(CdcrParams.LAST_PROCESSED_VERSION, lastProcessedVersion);
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- private void handleQueuesAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- NamedList hosts = new NamedList();
-
- for (CdcrReplicatorState state : replicatorManager.getReplicatorStates()) {
- NamedList queueStats = new NamedList();
-
- CdcrUpdateLog.CdcrLogReader logReader = state.getLogReader();
- if (logReader == null) {
- String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
- log.warn("The log reader for target collection {} is not initialised @ {}:{}",
- state.getTargetCollection(), collectionName, shard);
- queueStats.add(CdcrParams.QUEUE_SIZE, -1l);
- } else {
- queueStats.add(CdcrParams.QUEUE_SIZE, logReader.getNumberOfRemainingRecords());
- }
- queueStats.add(CdcrParams.LAST_TIMESTAMP, state.getTimestampOfLastProcessedOperation());
-
- if (hosts.get(state.getZkHost()) == null) {
- hosts.add(state.getZkHost(), new NamedList());
- }
- ((NamedList) hosts.get(state.getZkHost())).add(state.getTargetCollection(), queueStats);
- }
-
- rsp.add(CdcrParams.QUEUES, hosts);
- UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
- rsp.add(CdcrParams.TLOG_TOTAL_SIZE, updateLog.getTotalLogsSize());
- rsp.add(CdcrParams.TLOG_TOTAL_COUNT, updateLog.getTotalLogsNumber());
- rsp.add(CdcrParams.UPDATE_LOG_SYNCHRONIZER,
- updateLogSynchronizer.isStarted() ? CdcrParams.ProcessState.STARTED.toLower() : CdcrParams.ProcessState.STOPPED.toLower());
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- private void handleOpsAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- NamedList hosts = new NamedList();
-
- for (CdcrReplicatorState state : replicatorManager.getReplicatorStates()) {
- NamedList ops = new NamedList();
- ops.add(CdcrParams.COUNTER_ALL, state.getBenchmarkTimer().getOperationsPerSecond());
- ops.add(CdcrParams.COUNTER_ADDS, state.getBenchmarkTimer().getAddsPerSecond());
- ops.add(CdcrParams.COUNTER_DELETES, state.getBenchmarkTimer().getDeletesPerSecond());
-
- if (hosts.get(state.getZkHost()) == null) {
- hosts.add(state.getZkHost(), new NamedList());
- }
- ((NamedList) hosts.get(state.getZkHost())).add(state.getTargetCollection(), ops);
- }
-
- rsp.add(CdcrParams.OPERATIONS_PER_SECOND, hosts);
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- private void handleErrorsAction(SolrQueryRequest req, SolrQueryResponse rsp) {
- NamedList hosts = new NamedList();
-
- for (CdcrReplicatorState state : replicatorManager.getReplicatorStates()) {
- NamedList errors = new NamedList();
-
- errors.add(CdcrParams.CONSECUTIVE_ERRORS, state.getConsecutiveErrors());
- errors.add(CdcrReplicatorState.ErrorType.BAD_REQUEST.toLower(), state.getErrorCount(CdcrReplicatorState.ErrorType.BAD_REQUEST));
- errors.add(CdcrReplicatorState.ErrorType.INTERNAL.toLower(), state.getErrorCount(CdcrReplicatorState.ErrorType.INTERNAL));
-
- NamedList lastErrors = new NamedList();
- for (String[] lastError : state.getLastErrors()) {
- lastErrors.add(lastError[0], lastError[1]);
- }
- errors.add(CdcrParams.LAST, lastErrors);
-
- if (hosts.get(state.getZkHost()) == null) {
- hosts.add(state.getZkHost(), new NamedList());
- }
- ((NamedList) hosts.get(state.getZkHost())).add(state.getTargetCollection(), errors);
- }
-
- rsp.add(CdcrParams.ERRORS, hosts);
- }
-
- private void handleBootstrapAction(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, InterruptedException, SolrServerException {
- String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
- if (!leaderStateManager.amILeader()) {
- log.warn("Action {} sent to non-leader replica @ {}:{}", CdcrParams.CdcrAction.BOOTSTRAP, collectionName, shard);
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Action " + CdcrParams.CdcrAction.BOOTSTRAP +
- " sent to non-leader replica");
- }
- CountDownLatch latch = new CountDownLatch(1); // latch to make sure BOOTSTRAP_STATUS gives correct response
-
- Runnable runnable = () -> {
- Lock recoveryLock = req.getCore().getSolrCoreState().getRecoveryLock();
- boolean locked = recoveryLock.tryLock();
- SolrCoreState coreState = core.getSolrCoreState();
- try {
- if (!locked) {
- handleCancelBootstrap(req, rsp);
- } else if (leaderStateManager.amILeader()) {
- coreState.setCdcrBootstrapRunning(true);
- latch.countDown(); // free the latch as current bootstrap is executing
- //running.set(true);
- String leaderUrl = ReplicationHandler.getObjectWithBackwardCompatibility(req.getParams(), ReplicationHandler.LEADER_URL, ReplicationHandler.LEGACY_LEADER_URL, null);
- BootstrapCallable bootstrapCallable = new BootstrapCallable(leaderUrl, core);
- coreState.setCdcrBootstrapCallable(bootstrapCallable);
- Future<Boolean> bootstrapFuture = core.getCoreContainer().getUpdateShardHandler().getRecoveryExecutor()
- .submit(bootstrapCallable);
- coreState.setCdcrBootstrapFuture(bootstrapFuture);
- try {
- bootstrapFuture.get();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- log.warn("Bootstrap was interrupted", e);
- } catch (ExecutionException e) {
- log.error("Bootstrap operation failed", e);
- }
- } else {
- log.error("Action {} sent to non-leader replica @ {}:{}. Aborting bootstrap.", CdcrParams.CdcrAction.BOOTSTRAP, collectionName, shard);
- }
- } finally {
- if (locked) {
- coreState.setCdcrBootstrapRunning(false);
- recoveryLock.unlock();
- } else {
- latch.countDown(); // free the latch as current bootstrap is executing
- }
- }
- };
-
- try {
- core.getCoreContainer().getUpdateShardHandler().getUpdateExecutor().submit(runnable);
- rsp.add(RESPONSE_STATUS, "submitted");
- latch.await(10000, TimeUnit.MILLISECONDS); // put the latch for current bootstrap command
- } catch (RejectedExecutionException ree) {
- // no problem, we're probably shutting down
- rsp.add(RESPONSE_STATUS, "failed");
- }
- }
-
- private void handleCancelBootstrap(SolrQueryRequest req, SolrQueryResponse rsp) {
- BootstrapCallable callable = (BootstrapCallable)core.getSolrCoreState().getCdcrBootstrapCallable();
- IOUtils.closeQuietly(callable);
- rsp.add(RESPONSE_STATUS, "cancelled");
- }
-
- private void handleBootstrapStatus(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
- SolrCoreState coreState = core.getSolrCoreState();
- if (coreState.getCdcrBootstrapRunning()) {
- rsp.add(RESPONSE_STATUS, RUNNING);
- return;
- }
-
- Future<Boolean> future = coreState.getCdcrBootstrapFuture();
- BootstrapCallable callable = (BootstrapCallable)coreState.getCdcrBootstrapCallable();
- if (future == null) {
- rsp.add(RESPONSE_STATUS, "notfound");
- rsp.add(RESPONSE_MESSAGE, "No bootstrap found in running, completed or failed states");
- } else if (future.isCancelled() || callable.isClosed()) {
- rsp.add(RESPONSE_STATUS, "cancelled");
- } else if (future.isDone()) {
- // could be a normal termination or an exception
- try {
- Boolean result = future.get();
- if (result) {
- rsp.add(RESPONSE_STATUS, COMPLETED);
- } else {
- rsp.add(RESPONSE_STATUS, FAILED);
- }
- } catch (InterruptedException e) {
- // should not happen?
- } catch (ExecutionException e) {
- rsp.add(RESPONSE_STATUS, FAILED);
- rsp.add(RESPONSE, e);
- } catch (CancellationException ce) {
- rsp.add(RESPONSE_STATUS, FAILED);
- rsp.add(RESPONSE_MESSAGE, "Bootstrap was cancelled");
- }
- } else {
- rsp.add(RESPONSE_STATUS, RUNNING);
- }
- }
-
- static class BootstrapCallable implements Callable<Boolean>, Closeable {
- private final String leaderUrl;
- private final SolrCore core;
- private volatile boolean closed = false;
-
- BootstrapCallable(String leaderUrl, SolrCore core) {
- this.leaderUrl = leaderUrl;
- this.core = core;
- }
-
- @Override
- public void close() throws IOException {
- closed = true;
- SolrRequestHandler handler = core.getRequestHandler(ReplicationHandler.PATH);
- ReplicationHandler replicationHandler = (ReplicationHandler) handler;
- replicationHandler.abortFetch();
- }
-
- public boolean isClosed() {
- return closed;
- }
-
- @Override
- public Boolean call() throws Exception {
- boolean success = false;
- UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
- // we start buffering updates as a safeguard however we do not expect
- // to receive any updates from the source during bootstrap
- ulog.bufferUpdates();
- try {
- commitOnLeader(leaderUrl);
- // use rep handler directly, so we can do this sync rather than async
- SolrRequestHandler handler = core.getRequestHandler(ReplicationHandler.PATH);
- ReplicationHandler replicationHandler = (ReplicationHandler) handler;
-
- if (replicationHandler == null) {
- throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
- "Skipping recovery, no " + ReplicationHandler.PATH + " handler found");
- }
-
- ModifiableSolrParams solrParams = new ModifiableSolrParams();
- solrParams.set(ReplicationHandler.LEADER_URL, leaderUrl);
- // we do not want the raw tlog files from the source
- solrParams.set(ReplicationHandler.TLOG_FILES, false);
-
- success = replicationHandler.doFetch(solrParams, false).getSuccessful();
-
- Future<UpdateLog.RecoveryInfo> future = ulog.applyBufferedUpdates();
- if (future == null) {
- // no replay needed
- log.info("No replay needed.");
- } else {
- log.info("Replaying buffered documents.");
- // wait for replay
- UpdateLog.RecoveryInfo report = future.get();
- if (report.failed) {
- SolrException.log(log, "Replay failed");
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Replay failed");
- }
- }
- if (success) {
- ZkController zkController = core.getCoreContainer().getZkController();
- String collectionName = core.getCoreDescriptor().getCollectionName();
- ClusterState clusterState = zkController.getZkStateReader().getClusterState();
- DocCollection collection = clusterState.getCollection(collectionName);
- Slice slice = collection.getSlice(core.getCoreDescriptor().getCloudDescriptor().getShardId());
- ZkShardTerms terms = zkController.getShardTerms(collectionName, slice.getName());
- String coreNodeName = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
- Set<String> allExceptLeader = slice.getReplicas().stream().filter(replica -> !replica.getName().equals(coreNodeName)).map(Replica::getName).collect(Collectors.toSet());
- terms.ensureTermsIsHigher(coreNodeName, allExceptLeader);
- }
- return success;
- } finally {
- if (closed || !success) {
- // we cannot apply the buffer in this case because it will introduce newer versions in the
- // update log and then the source cluster will get those versions via collectioncheckpoint
- // causing the versions in between to be completely missed
- boolean dropped = ulog.dropBufferedUpdates();
- assert dropped;
- }
- }
- }
-
- private void commitOnLeader(String leaderUrl) throws SolrServerException,
- IOException {
- try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
- .withConnectionTimeout(30000)
- .build()) {
- UpdateRequest ureq = new UpdateRequest();
- ureq.setParams(new ModifiableSolrParams());
- ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
- ureq.getParams().set(UpdateParams.OPEN_SEARCHER, false);
- ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true).process(
- client);
- }
- }
- }
-
- @Override
- public String getDescription() {
- return "Manage Cross Data Center Replication";
- }
-
- @Override
- public Category getCategory() {
- return Category.REPLICATION;
- }
-
- /**
- * A thread subclass for executing a single
- * {@link org.apache.solr.handler.CdcrParams.CdcrAction#SHARDCHECKPOINT} action.
- */
- private static final class SliceCheckpointCallable implements Callable<Long> {
-
- final String baseUrl;
- final String cdcrPath;
-
- SliceCheckpointCallable(final String baseUrl, final String cdcrPath) {
- this.baseUrl = baseUrl;
- this.cdcrPath = cdcrPath;
- }
-
- @Override
- public Long call() throws Exception {
- try (HttpSolrClient server = new HttpSolrClient.Builder(baseUrl)
- .withConnectionTimeout(15000)
- .withSocketTimeout(60000)
- .build()) {
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.SHARDCHECKPOINT.toString());
-
- @SuppressWarnings({"rawtypes"})
- SolrRequest request = new QueryRequest(params);
- request.setPath(cdcrPath);
-
- @SuppressWarnings({"rawtypes"})
- NamedList response = server.request(request);
- return (Long) response.get(CdcrParams.CHECKPOINT);
- }
- }
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrStateManager.java
deleted file mode 100644
index 151615e..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrStateManager.java
+++ /dev/null
@@ -1,47 +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 java.util.ArrayList;
-import java.util.List;
-
-/**
- * A state manager which implements an observer pattern to notify observers
- * of a state change.
- */
-abstract class CdcrStateManager {
-
- private List<CdcrStateObserver> observers = new ArrayList<>();
-
- void register(CdcrStateObserver observer) {
- this.observers.add(observer);
- }
-
- void callback() {
- for (CdcrStateObserver observer : observers) {
- observer.stateUpdate();
- }
- }
-
- interface CdcrStateObserver {
-
- void stateUpdate();
-
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
deleted file mode 100644
index 52590ee..0000000
--- a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
+++ /dev/null
@@ -1,192 +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 java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.update.CdcrUpdateLog;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * Synchronize periodically the update log of non-leader nodes with their leaders.
- * </p>
- * <p>
- * Non-leader nodes must always buffer updates in case of leader failures. They have to periodically
- * synchronize their update logs with their leader to remove old transaction logs that will never be used anymore.
- * This is performed by a background thread that is scheduled with a fixed delay. The background thread is sending
- * the action {@link org.apache.solr.handler.CdcrParams.CdcrAction#LASTPROCESSEDVERSION} to the leader to retrieve
- * the lowest last version number processed. This version is then used to move forward the buffer log reader.
- * </p>
- */
-class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
-
- private CdcrLeaderStateManager leaderStateManager;
- private ScheduledExecutorService scheduler;
-
- private final SolrCore core;
- private final String collection;
- private final String shardId;
- private final String path;
-
- private int timeSchedule = DEFAULT_TIME_SCHEDULE;
-
- private static final int DEFAULT_TIME_SCHEDULE = 60000; // by default, every minute
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- CdcrUpdateLogSynchronizer(SolrCore core, String path, SolrParams updateLogSynchonizerConfiguration) {
- this.core = core;
- this.path = path;
- this.collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
- this.shardId = core.getCoreDescriptor().getCloudDescriptor().getShardId();
- if (updateLogSynchonizerConfiguration != null) {
- this.timeSchedule = updateLogSynchonizerConfiguration.getInt(CdcrParams.SCHEDULE_PARAM, DEFAULT_TIME_SCHEDULE);
- }
- }
-
- void setLeaderStateManager(final CdcrLeaderStateManager leaderStateManager) {
- this.leaderStateManager = leaderStateManager;
- this.leaderStateManager.register(this);
- }
-
- @Override
- public void stateUpdate() {
- // If I am not the leader, I need to synchronise periodically my update log with my leader.
- if (!leaderStateManager.amILeader()) {
- scheduler = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-update-log-synchronizer"));
- scheduler.scheduleWithFixedDelay(new UpdateLogSynchronisation(), 0, timeSchedule, TimeUnit.MILLISECONDS);
- return;
- }
-
- this.shutdown();
- }
-
- boolean isStarted() {
- return scheduler != null;
- }
-
- void shutdown() {
- if (scheduler != null) {
- // interrupts are often dangerous in Lucene / Solr code, but the
- // test for this will leak threads without
- scheduler.shutdownNow();
- scheduler = null;
- }
- }
-
- private class UpdateLogSynchronisation implements Runnable {
-
- private String getLeaderUrl() {
- ZkController zkController = core.getCoreContainer().getZkController();
- ClusterState cstate = zkController.getClusterState();
- DocCollection docCollection = cstate.getCollection(collection);
- ZkNodeProps leaderProps = docCollection.getLeader(shardId);
- if (leaderProps == null) { // we might not have a leader yet, returns null
- return null;
- }
- ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
- return nodeProps.getCoreUrl();
- }
-
- @Override
- public void run() {
- try {
- String leaderUrl = getLeaderUrl();
- if (leaderUrl == null) { // we might not have a leader yet, stop and try again later
- return;
- }
-
- HttpSolrClient server = new HttpSolrClient.Builder(leaderUrl)
- .withConnectionTimeout(15000)
- .withSocketTimeout(60000)
- .build();
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.LASTPROCESSEDVERSION.toString());
-
- @SuppressWarnings({"rawtypes"})
- SolrRequest request = new QueryRequest(params);
- request.setPath(path);
-
- long lastVersion;
- try {
- @SuppressWarnings({"rawtypes"})
- NamedList response = server.request(request);
- lastVersion = (Long) response.get(CdcrParams.LAST_PROCESSED_VERSION);
- if (log.isDebugEnabled()) {
- log.debug("My leader {} says its last processed _version_ number is: {}. I am {}", leaderUrl, lastVersion,
- core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
- }
- } catch (IOException | SolrServerException e) {
- log.warn("Couldn't get last processed version from leader {}: ", leaderUrl, e);
- return;
- } finally {
- try {
- server.close();
- } catch (IOException ioe) {
- log.warn("Caught exception trying to close client to {}: ", leaderUrl, ioe);
- }
- }
-
- // if we received -1, it means that the log reader on the leader has not yet started to read log entries
- // do nothing
- if (lastVersion == -1) {
- return;
- }
-
- try {
- CdcrUpdateLog ulog = (CdcrUpdateLog) core.getUpdateHandler().getUpdateLog();
- if (ulog.isBuffering()) {
- log.debug("Advancing replica buffering tlog reader to {} @ {}:{}", lastVersion, collection, shardId);
- ulog.getBufferToggle().seek(lastVersion);
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- log.warn("Couldn't advance replica buffering tlog reader to {} (to remove old tlogs): ", lastVersion, e);
- } catch (IOException e) {
- log.warn("Couldn't advance replica buffering tlog reader to {} (to remove old tlogs): ", lastVersion, e);
- }
- } catch (Throwable e) {
- log.warn("Caught unexpected exception", e);
- throw e;
- }
- }
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 57f8ba0..2bd2045 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -93,10 +93,7 @@ import org.apache.solr.core.SolrCore;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.CommitUpdateCommand;
-import org.apache.solr.update.UpdateLog;
-import org.apache.solr.update.VersionInfo;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.FileUtils;
import org.apache.solr.util.PropertiesOutputStream;
@@ -137,14 +134,10 @@ public class IndexFetcher {
private volatile List<Map<String, Object>> confFilesToDownload;
- private volatile List<Map<String, Object>> tlogFilesToDownload;
-
private volatile List<Map<String, Object>> filesDownloaded;
private volatile List<Map<String, Object>> confFilesDownloaded;
- private volatile List<Map<String, Object>> tlogFilesDownloaded;
-
private volatile Map<String, Object> currentFile;
private volatile DirectoryFileFetcher dirFileFetcher;
@@ -263,10 +256,6 @@ public class IndexFetcher {
soTimeout = getParameter(initArgs, HttpClientUtil.PROP_SO_TIMEOUT, 120000, null);
}
- if (initArgs.getBooleanArg(TLOG_FILES) != null) {
- downloadTlogFiles = initArgs.getBooleanArg(TLOG_FILES);
- }
-
String httpBasicAuthUser = (String) initArgs.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
String httpBasicAuthPassword = (String) initArgs.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
myHttpClient = createHttpClient(solrCore, httpBasicAuthUser, httpBasicAuthPassword, useExternalCompression);
@@ -314,7 +303,6 @@ public class IndexFetcher {
private void fetchFileList(long gen) throws IOException {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(COMMAND, CMD_GET_FILE_LIST);
- params.set(TLOG_FILES, downloadTlogFiles);
params.set(GENERATION, String.valueOf(gen));
params.set(CommonParams.WT, JAVABIN);
params.set(CommonParams.QT, ReplicationHandler.PATH);
@@ -340,11 +328,6 @@ public class IndexFetcher {
files = (List<Map<String,Object>>) response.get(CONF_FILES);
if (files != null)
confFilesToDownload = Collections.synchronizedList(files);
-
- files = (List<Map<String, Object>>) response.get(TLOG_FILES);
- if (files != null) {
- tlogFilesToDownload = Collections.synchronizedList(files);
- }
} catch (SolrServerException e) {
throw new IOException(e);
}
@@ -500,11 +483,6 @@ public class IndexFetcher {
if (log.isInfoEnabled()) {
log.info("Number of files in latest index in leader: {}", filesToDownload.size());
}
- if (tlogFilesToDownload != null) {
- if (log.isInfoEnabled()) {
- log.info("Number of tlog files in leader: {}", tlogFilesToDownload.size());
- }
- }
// Create the sync service
fsyncService = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("fsyncService"));
@@ -522,11 +500,6 @@ public class IndexFetcher {
tmpIndexDir = solrCore.getDirectoryFactory().get(tmpIndexDirPath, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
- // tmp dir for tlog files
- if (tlogFilesToDownload != null) {
- tmpTlogDir = new File(solrCore.getUpdateHandler().getUpdateLog().getLogDir(), "tlog." + timestamp);
- }
-
// cindex dir...
indexDirPath = solrCore.getIndexDir();
indexDir = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
@@ -588,10 +561,6 @@ public class IndexFetcher {
long bytesDownloaded = downloadIndexFiles(isFullCopyNeeded, indexDir,
tmpIndexDir, indexDirPath, tmpIndexDirPath, latestGeneration);
- if (tlogFilesToDownload != null) {
- bytesDownloaded += downloadTlogFiles(tmpTlogDir, latestGeneration);
- reloadCore = true; // reload update log
- }
final long timeTakenSeconds = getReplicationTimeElapsed();
final Long bytesDownloadedPerSecond = (timeTakenSeconds != 0 ? Long.valueOf(bytesDownloaded / timeTakenSeconds) : null);
log.info("Total time taken for download (fullCopy={},bytesDownloaded={}) : {} secs ({} bytes/sec) to {}",
@@ -607,10 +576,6 @@ public class IndexFetcher {
} else {
successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
}
- if (tlogFilesToDownload != null) {
- // move tlog files and refresh ulog only if we successfully installed a new index
- successfulInstall &= moveTlogFiles(tmpTlogDir);
- }
if (successfulInstall) {
if (isFullCopyNeeded) {
// let the system know we are changing dir's and the old one
@@ -637,10 +602,6 @@ public class IndexFetcher {
} else {
successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
}
- if (tlogFilesToDownload != null) {
- // move tlog files and refresh ulog only if we successfully installed a new index
- successfulInstall &= moveTlogFiles(tmpTlogDir);
- }
if (successfulInstall) {
logReplicationTimeAndConfFiles(modifiedConfFiles,
successfulInstall);
@@ -733,7 +694,7 @@ public class IndexFetcher {
core.getUpdateHandler().getSolrCoreState().setLastReplicateIndexSuccess(successfulInstall);
}
- filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = tlogFilesToDownload = tlogFilesDownloaded = null;
+ filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = null;
markReplicationStop();
dirFileFetcher = null;
localFileFetcher = null;
@@ -990,30 +951,6 @@ public class IndexFetcher {
}
/**
- * Download all the tlog files to the temp tlog directory.
- */
- private long downloadTlogFiles(File tmpTlogDir, long latestGeneration) throws Exception {
- log.info("Starting download of tlog files from leader: {}", tlogFilesToDownload);
- tlogFilesDownloaded = Collections.synchronizedList(new ArrayList<>());
- long bytesDownloaded = 0;
-
- boolean status = tmpTlogDir.mkdirs();
- if (!status) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Failed to create temporary tlog folder: " + tmpTlogDir.getName());
- }
- for (Map<String, Object> file : tlogFilesToDownload) {
- String saveAs = (String) (file.get(ALIAS) == null ? file.get(NAME) : file.get(ALIAS));
- localFileFetcher = new LocalFsFileFetcher(tmpTlogDir, file, saveAs, TLOG_FILE, latestGeneration);
- currentFile = file;
- localFileFetcher.fetchFile();
- bytesDownloaded += localFileFetcher.getBytesDownloaded();
- tlogFilesDownloaded.add(new HashMap<>(file));
- }
- return bytesDownloaded;
- }
-
- /**
* Download the index files. If a new index is needed, download all the files.
*
* @param downloadCompleteIndex is it a fresh index copy
@@ -1340,50 +1277,6 @@ public class IndexFetcher {
}
/**
- * <p>
- * Copy all the tlog files from the temp tlog dir to the actual tlog dir, and reset
- * the {@link UpdateLog}. The copy will try to preserve the original tlog directory
- * if the copy fails.
- * </p>
- * <p>
- * This assumes that the tlog files transferred from the leader are in synch with the
- * index files transferred from the leader. The reset of the update log relies on the version
- * of the latest operations found in the tlog files. If the tlogs are ahead of the latest commit
- * point, it will not copy all the needed buffered updates for the replay and it will miss
- * some operations.
- * </p>
- */
- private boolean moveTlogFiles(File tmpTlogDir) {
- UpdateLog ulog = solrCore.getUpdateHandler().getUpdateLog();
-
- VersionInfo vinfo = ulog.getVersionInfo();
- vinfo.blockUpdates(); // block updates until the new update log is initialised
- try {
- // reset the update log before copying the new tlog directory
- CdcrUpdateLog.BufferedUpdates bufferedUpdates = ((CdcrUpdateLog) ulog).resetForRecovery();
- // try to move the temp tlog files to the tlog directory
- if (!copyTmpTlogFiles2Tlog(tmpTlogDir)) return false;
- // reinitialise the update log and copy the buffered updates
- if (bufferedUpdates.tlog != null) {
- // map file path to its new backup location
- File parentDir = FileSystems.getDefault().getPath(solrCore.getUpdateHandler().getUpdateLog().getLogDir()).getParent().toFile();
- File backupTlogDir = new File(parentDir, tmpTlogDir.getName());
- bufferedUpdates.tlog = new File(backupTlogDir, bufferedUpdates.tlog.getName());
- }
- // init the update log with the new set of tlog files, and copy the buffered updates
- ((CdcrUpdateLog) ulog).initForRecovery(bufferedUpdates.tlog, bufferedUpdates.offset);
- }
- catch (Exception e) {
- log.error("Unable to copy tlog files", e);
- return false;
- }
- finally {
- vinfo.unblockUpdates();
- }
- return true;
- }
-
- /**
* Make file list
*/
private List<File> makeTmpConfDirFileList(File dir, List<File> fileList) {
@@ -1570,20 +1463,7 @@ public class IndexFetcher {
return timeElapsed;
}
- List<Map<String, Object>> getTlogFilesToDownload() {
- //make a copy first because it can be null later
- List<Map<String, Object>> tmp = tlogFilesToDownload;
- //create a new instance. or else iterator may fail
- return tmp == null ? Collections.emptyList() : new ArrayList<>(tmp);
- }
-
- List<Map<String, Object>> getTlogFilesDownloaded() {
- //make a copy first because it can be null later
- List<Map<String, Object>> tmp = tlogFilesDownloaded;
- // NOTE: it's safe to make a copy of a SynchronizedCollection(ArrayList)
- return tmp == null ? Collections.emptyList() : new ArrayList<>(tmp);
- }
-
+ @SuppressWarnings({"unchecked"})
List<Map<String, Object>> getConfFilesToDownload() {
//make a copy first because it can be null later
List<Map<String, Object>> tmp = confFilesToDownload;
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 1d8bdb9..1cf89a9 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -96,7 +96,6 @@ import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.SolrIndexWriter;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.common.util.SolrNamedThreadFactory;
@@ -731,19 +730,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
rsp.add(CMD_GET_FILE_LIST, result);
- if (solrParams.getBool(TLOG_FILES, false)) {
- try {
- List<Map<String, Object>> tlogfiles = getTlogFileList(commit);
- log.info("Adding tlog files to list: {}", tlogfiles);
- rsp.add(TLOG_FILES, tlogfiles);
- }
- catch (IOException e) {
- log.error("Unable to get tlog file names for indexCommit generation: {}", commit.getGeneration(), e);
- reportErrorOnResponse(rsp, "unable to get tlog file names for given index generation", e);
- return;
- }
- }
-
if (confFileNameAlias.size() < 1 || core.getCoreContainer().isZooKeeperAware())
return;
log.debug("Adding config files to list: {}", includeConfFiles);
@@ -763,29 +749,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
/**
- * Retrieves the list of tlog files associated to a commit point.
- * NOTE: The commit <b>MUST</b> be reserved before calling this method
- */
- List<Map<String, Object>> getTlogFileList(IndexCommit commit) throws IOException {
- long maxVersion = this.getMaxVersion(commit);
- CdcrUpdateLog ulog = (CdcrUpdateLog) core.getUpdateHandler().getUpdateLog();
- String[] logList = ulog.getLogList(new File(ulog.getLogDir()));
- List<Map<String, Object>> tlogFiles = new ArrayList<>();
- for (String fileName : logList) {
- // filter out tlogs that are older than the current index commit generation, so that the list of tlog files is
- // in synch with the latest index commit point
- long startVersion = Math.abs(Long.parseLong(fileName.substring(fileName.lastIndexOf('.') + 1)));
- if (startVersion < maxVersion) {
- Map<String, Object> fileMeta = new HashMap<>();
- fileMeta.put(NAME, fileName);
- fileMeta.put(SIZE, new File(ulog.getLogDir(), fileName).length());
- tlogFiles.add(fileMeta);
- }
- }
- return tlogFiles;
- }
-
- /**
* Retrieves the maximum version number from an index commit.
* NOTE: The commit <b>MUST</b> be reserved before calling this method
*/
@@ -1873,8 +1836,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
public static final String CONF_FILES = "confFiles";
- public static final String TLOG_FILES = "tlogFiles";
-
public static final String REPLICATE_AFTER = "replicateAfter";
public static final String FILE_STREAM = "filestream";
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index 3568a54..7956143 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -82,7 +82,6 @@ import org.apache.solr.search.SolrDocumentFetcher;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SolrReturnFields;
import org.apache.solr.search.SyntaxError;
-import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.DocumentBuilder;
import org.apache.solr.update.IndexFingerprint;
import org.apache.solr.update.PeerSync;
@@ -269,11 +268,7 @@ public class RealTimeGetComponent extends SearchComponent
if (oper == UpdateLog.ADD) {
doc = toSolrDoc((SolrInputDocument)entry.get(entry.size()-1), core.getLatestSchema());
} else if (oper == UpdateLog.UPDATE_INPLACE) {
- if (ulog instanceof CdcrUpdateLog) {
- assert entry.size() == 6;
- } else {
- assert entry.size() == 5;
- }
+ assert entry.size() == 5;
// For in-place update case, we have obtained the partial document till now. We need to
// resolve it to a full document to be returned to the user.
doc = resolveFullDocument(core, idBytes.get(), rsp.getReturnFields(), (SolrInputDocument)entry.get(entry.size()-1), entry, null);
@@ -571,12 +566,8 @@ public class RealTimeGetComponent extends SearchComponent
}
switch (oper) {
case UpdateLog.UPDATE_INPLACE:
- if (ulog instanceof CdcrUpdateLog) {
- assert entry.size() == 6;
- } else {
- assert entry.size() == 5;
- }
-
+ assert entry.size() == 5;
+
if (resolveFullDocument) {
SolrInputDocument doc = (SolrInputDocument)entry.get(entry.size()-1);
try {
diff --git a/solr/core/src/java/org/apache/solr/update/CdcrTransactionLog.java b/solr/core/src/java/org/apache/solr/update/CdcrTransactionLog.java
deleted file mode 100644
index 86cee71..0000000
--- a/solr/core/src/java/org/apache/solr/update/CdcrTransactionLog.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.update;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.lang.invoke.MethodHandles;
-import java.nio.channels.Channels;
-import java.nio.file.Files;
-import java.util.Collection;
-
-import org.apache.lucene.util.BytesRef;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.util.FastOutputStream;
-import org.apache.solr.common.util.JavaBinCodec;
-import org.apache.solr.common.util.ObjectReleaseTracker;
-import org.apache.solr.update.processor.CdcrUpdateProcessor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Extends {@link org.apache.solr.update.TransactionLog} to:
- * <ul>
- * <li>reopen automatically the output stream if its reference count reached 0. This is achieved by extending
- * methods {@link #incref()}, {@link #close()} and {@link #reopenOutputStream()}.</li>
- * <li>encode the number of records in the tlog file in the last commit record. The number of records will be
- * decoded and reuse if the tlog file is reopened. This is achieved by extending the constructor, and the
- * methods {@link #writeCommit(CommitUpdateCommand)} and {@link #getReader(long)}.</li>
- * </ul>
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-public class CdcrTransactionLog extends TransactionLog {
-
- private boolean isReplaying;
- long startVersion; // (absolute) version of the first element of this transaction log
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private boolean debug = log.isDebugEnabled();
-
- CdcrTransactionLog(File tlogFile, Collection<String> globalStrings) {
- super(tlogFile, globalStrings);
-
- // The starting version number will be used to seek more efficiently tlogs
- // and to filter out tlog files during replication (in ReplicationHandler#getTlogFileList)
- String filename = tlogFile.getName();
- startVersion = Math.abs(Long.parseLong(filename.substring(filename.lastIndexOf('.') + 1)));
-
- isReplaying = false;
- }
-
- CdcrTransactionLog(File tlogFile, Collection<String> globalStrings, boolean openExisting) {
- super(tlogFile, globalStrings, openExisting);
-
- // The starting version number will be used to seek more efficiently tlogs
- String filename = tlogFile.getName();
- startVersion = Math.abs(Long.parseLong(filename.substring(filename.lastIndexOf('.') + 1)));
-
- numRecords = openExisting ? this.readNumRecords() : 0;
- // if we try to reopen an existing tlog file and that the number of records is equal to 0, then we are replaying
- // the log and we will append a commit
- if (openExisting && numRecords == 0) {
- isReplaying = true;
- }
- }
-
- /**
- * Returns the number of records in the log (currently includes the header and an optional commit).
- */
- public int numRecords() {
- return super.numRecords();
- }
-
- /**
- * The last record of the transaction log file is expected to be a commit with a 4 byte integer that encodes the
- * number of records in the file.
- */
- private int readNumRecords() {
- try {
- if (endsWithCommit()) {
- long size = fos.size();
- // 4 bytes for the record size, the lenght of the end message + 1 byte for its value tag,
- // and 4 bytes for the number of records
- long pos = size - 4 - END_MESSAGE.length() - 1 - 4;
- if (pos < 0) return 0;
- try (ChannelFastInputStream is = new ChannelFastInputStream(channel, pos)) {
- return is.readInt();
- }
- }
- } catch (IOException e) {
- log.error("Error while reading number of records in tlog {}", this, e);
- }
- return 0;
- }
-
- @Override
- public long write(AddUpdateCommand cmd, long prevPointer) {
- assert (-1 <= prevPointer && (cmd.isInPlaceUpdate() || (-1 == prevPointer)));
-
- LogCodec codec = new LogCodec(resolver);
- SolrInputDocument sdoc = cmd.getSolrInputDocument();
-
- try {
- checkWriteHeader(codec, sdoc);
-
- // adaptive buffer sizing
- int bufSize = lastAddSize; // unsynchronized access of lastAddSize should be fine
- bufSize = Math.min(1024*1024, bufSize+(bufSize>>3)+256);
-
- MemOutputStream out = new MemOutputStream(new byte[bufSize]);
- codec.init(out);
- if (cmd.isInPlaceUpdate()) {
- codec.writeTag(JavaBinCodec.ARR, 6);
- codec.writeInt(UpdateLog.UPDATE_INPLACE); // should just take one byte
- codec.writeLong(cmd.getVersion());
- codec.writeLong(prevPointer);
- codec.writeLong(cmd.prevVersion);
- if (cmd.getReq().getParamString().contains(CdcrUpdateProcessor.CDCR_UPDATE)) {
- // if the update is received via cdcr source; add boolean entry
- // CdcrReplicator.isTargetCluster() checks that particular boolean to accept or discard the update
- // to forward to its own target cluster
- codec.writePrimitive(true);
- } else {
- codec.writePrimitive(false);
- }
- codec.writeSolrInputDocument(cmd.getSolrInputDocument());
-
- } else {
- codec.writeTag(JavaBinCodec.ARR, 4);
- codec.writeInt(UpdateLog.ADD); // should just take one byte
- codec.writeLong(cmd.getVersion());
- if (cmd.getReq().getParamString().contains(CdcrUpdateProcessor.CDCR_UPDATE)) {
- // if the update is received via cdcr source; add extra boolean entry
- // CdcrReplicator.isTargetCluster() checks that particular boolean to accept or discard the update
- // to forward to its own target cluster
- codec.writePrimitive(true);
- } else {
- codec.writePrimitive(false);
- }
- codec.writeSolrInputDocument(cmd.getSolrInputDocument());
- }
- lastAddSize = (int)out.size();
-
- synchronized (this) {
- long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
- assert pos != 0;
-
- /***
- System.out.println("###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
- if (pos != fos.size()) {
- throw new RuntimeException("ERROR" + "###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
- }
- ***/
-
- out.writeAll(fos);
- endRecord(pos);
- // fos.flushBuffer(); // flush later
- return pos;
- }
-
- } catch (IOException e) {
- // TODO: reset our file pointer back to "pos", the start of this record.
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error logging add", e);
- }
- }
-
- @Override
- public long writeDelete(DeleteUpdateCommand cmd) {
- LogCodec codec = new LogCodec(resolver);
-
- try {
- checkWriteHeader(codec, null);
-
- BytesRef br = cmd.getIndexedId();
-
- MemOutputStream out = new MemOutputStream(new byte[20 + br.length]);
- codec.init(out);
- codec.writeTag(JavaBinCodec.ARR, 4);
- codec.writeInt(UpdateLog.DELETE); // should just take one byte
- codec.writeLong(cmd.getVersion());
- codec.writeByteArray(br.bytes, br.offset, br.length);
- if (cmd.getReq().getParamString().contains(CdcrUpdateProcessor.CDCR_UPDATE)) {
- // if the update is received via cdcr source; add extra boolean entry
- // CdcrReplicator.isTargetCluster() checks that particular boolean to accept or discard the update
- // to forward to its own target cluster
- codec.writePrimitive(true);
- } else {
- codec.writePrimitive(false);
- }
-
- synchronized (this) {
- long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
- assert pos != 0;
- out.writeAll(fos);
- endRecord(pos);
- // fos.flushBuffer(); // flush later
- return pos;
- }
-
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- }
-
- @Override
- public long writeDeleteByQuery(DeleteUpdateCommand cmd) {
- LogCodec codec = new LogCodec(resolver);
- try {
- checkWriteHeader(codec, null);
-
- MemOutputStream out = new MemOutputStream(new byte[20 + (cmd.query.length())]);
- codec.init(out);
- codec.writeTag(JavaBinCodec.ARR, 4);
- codec.writeInt(UpdateLog.DELETE_BY_QUERY); // should just take one byte
- codec.writeLong(cmd.getVersion());
- codec.writeStr(cmd.query);
- if (cmd.getReq().getParamString().contains(CdcrUpdateProcessor.CDCR_UPDATE)) {
- // if the update is received via cdcr source; add extra boolean entry
- // CdcrReplicator.isTargetCluster() checks that particular boolean to accept or discard the update
- // to forward to its own target cluster
- codec.writePrimitive(true);
- } else {
- codec.writePrimitive(false);
- }
- synchronized (this) {
- long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
- out.writeAll(fos);
- endRecord(pos);
- // fos.flushBuffer(); // flush later
- return pos;
- }
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- }
-
- @Override
- public long writeCommit(CommitUpdateCommand cmd) {
- LogCodec codec = new LogCodec(resolver);
- synchronized (this) {
- try {
- long pos = fos.size(); // if we had flushed, this should be equal to channel.position()
-
- if (pos == 0) {
- writeLogHeader(codec);
- pos = fos.size();
- }
- codec.init(fos);
- codec.writeTag(JavaBinCodec.ARR, 4);
- codec.writeInt(UpdateLog.COMMIT); // should just take one byte
- codec.writeLong(cmd.getVersion());
- codec.writeTag(JavaBinCodec.INT); // Enforce the encoding of a plain integer, to simplify decoding
- fos.writeInt(numRecords + 1); // the number of records in the file - +1 to account for the commit operation being written
- codec.writeStr(END_MESSAGE); // ensure these bytes are (almost) last in the file
-
- endRecord(pos);
-
- fos.flush(); // flush since this will be the last record in a log fill
- assert fos.size() == channel.size();
-
- isReplaying = false; // we have replayed and appended a commit record with the number of records in the file
-
- return pos;
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- }
- }
-
- /**
- * Returns a reader that can be used while a log is still in use.
- * Currently only *one* LogReader may be outstanding, and that log may only
- * be used from a single thread.
- */
- @Override
- public LogReader getReader(long startingPos) {
- return new CdcrLogReader(startingPos);
- }
-
- public class CdcrLogReader extends LogReader {
-
- private int numRecords = 1; // start at 1 to account for the header record
-
- public CdcrLogReader(long startingPos) {
- super(startingPos);
- }
-
- @Override
- public Object next() throws IOException, InterruptedException {
- Object o = super.next();
- if (o != null) {
- this.numRecords++;
- // We are replaying the log. We need to update the number of records for the writeCommit.
- if (isReplaying) {
- synchronized (CdcrTransactionLog.this) {
- CdcrTransactionLog.this.numRecords = this.numRecords;
- }
- }
- }
- return o;
- }
-
- }
-
- @Override
- public void incref() {
- // if the refcount is 0, we need to reopen the output stream
- if (refcount.getAndIncrement() == 0) {
- reopenOutputStream(); // synchronised with this
- }
- }
-
- /**
- * Modified to act like {@link #incref()} in order to be compatible with {@link UpdateLog#recoverFromLog()}.
- * Otherwise, we would have to duplicate the method {@link UpdateLog#recoverFromLog()} in
- * {@link org.apache.solr.update.CdcrUpdateLog} and change the call
- * {@code if (!ll.try_incref()) continue; } to {@code incref(); }.
- */
- @Override
- public boolean try_incref() {
- this.incref();
- return true;
- }
-
- @Override
- public void close() {
- try {
- if (debug) {
- log.debug("Closing tlog {}", this);
- }
-
- synchronized (this) {
- if (fos != null) {
- fos.flush();
- fos.close();
-
- // dereference these variables for GC
- fos = null;
- os = null;
- channel = null;
- raf = null;
- }
- }
-
- if (deleteOnClose) {
- try {
- Files.deleteIfExists(tlogFile.toPath());
- } catch (IOException e) {
- // TODO: should this class care if a file couldnt be deleted?
- // this just emulates previous behavior, where only SecurityException would be handled.
- }
- }
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- } finally {
- assert ObjectReleaseTracker.release(this);
- }
- }
-
- /**
- * Re-open the output stream of the tlog and position
- * the file pointer at the end of the file. It assumes
- * that the tlog is non-empty and that the tlog's header
- * has been already read.
- */
- synchronized void reopenOutputStream() {
- try {
- if (debug) {
- log.debug("Re-opening tlog's output stream: {}", this);
- }
-
- raf = new RandomAccessFile(this.tlogFile, "rw");
- channel = raf.getChannel();
- long start = raf.length();
- raf.seek(start);
- os = Channels.newOutputStream(channel);
- fos = new FastOutputStream(os, new byte[65536], 0);
- fos.setWritten(start); // reflect that we aren't starting at the beginning
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/update/CdcrUpdateLog.java b/solr/core/src/java/org/apache/solr/update/CdcrUpdateLog.java
deleted file mode 100644
index eee3127..0000000
--- a/solr/core/src/java/org/apache/solr/update/CdcrUpdateLog.java
+++ /dev/null
@@ -1,796 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.update;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingDeque;
-
-import org.apache.lucene.util.BytesRef;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.request.LocalSolrQueryRequest;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.update.processor.DistributedUpdateProcessor;
-import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An extension of the {@link org.apache.solr.update.UpdateLog} for the CDCR scenario.<br>
- * Compared to the original update log implementation, transaction logs are removed based on
- * pointers instead of a fixed size limit. Pointers are created by the CDC replicators and
- * correspond to replication checkpoints. If all pointers are ahead of a transaction log,
- * this transaction log is removed.<br>
- * Given that the number of transaction logs can become considerable if some pointers are
- * lagging behind, the {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader} provides
- * a {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader#seek(long)} method to
- * efficiently lookup a particular transaction log file given a version number.
- */
-public class CdcrUpdateLog extends UpdateLog {
-
- protected final Map<CdcrLogReader, CdcrLogPointer> logPointers = new ConcurrentHashMap<>();
-
- /**
- * A reader that will be used as toggle to turn on/off the buffering of tlogs
- */
- private CdcrLogReader bufferToggle;
-
- public static String LOG_FILENAME_PATTERN = "%s.%019d.%1d";
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private boolean debug = log.isDebugEnabled();
-
- @Override
- public void init(UpdateHandler uhandler, SolrCore core) {
- // remove dangling readers
- for (CdcrLogReader reader : logPointers.keySet()) {
- reader.close();
- }
- logPointers.clear();
-
- // init
- super.init(uhandler, core);
- }
-
- @Override
- public TransactionLog newTransactionLog(File tlogFile, Collection<String> globalStrings, boolean openExisting) {
- return new CdcrTransactionLog(tlogFile, globalStrings, openExisting);
- }
-
- @Override
- protected void addOldLog(TransactionLog oldLog, boolean removeOld) {
- if (oldLog == null) return;
-
- numOldRecords += oldLog.numRecords();
-
- int currRecords = numOldRecords;
-
- if (oldLog != tlog && tlog != null) {
- currRecords += tlog.numRecords();
- }
-
- while (removeOld && logs.size() > 0) {
- TransactionLog log = logs.peekLast();
- int nrec = log.numRecords();
-
- // remove oldest log if we don't need it to keep at least numRecordsToKeep, or if
- // we already have the limit of 10 log files.
- if (currRecords - nrec >= numRecordsToKeep || logs.size() >= 10) {
- // remove the oldest log if nobody points to it
- if (!this.hasLogPointer(log)) {
- currRecords -= nrec;
- numOldRecords -= nrec;
- TransactionLog last = logs.removeLast();
- last.deleteOnClose = true;
- last.close(); // it will be deleted if no longer in use
- continue;
- }
- // we have one log with one pointer, we should stop removing logs
- break;
- }
-
- break;
- }
-
- // Decref old log as we do not write to it anymore
- // If the oldlog is uncapped, i.e., a write commit has to be performed
- // during recovery, the output stream will be automatically re-open when
- // TransaactionLog#incref will be called.
- oldLog.deleteOnClose = false;
- oldLog.decref();
-
- // don't incref... we are taking ownership from the caller.
- logs.addFirst(oldLog);
- }
-
- /**
- * Checks if one of the log pointer is pointing to the given tlog.
- */
- private boolean hasLogPointer(TransactionLog tlog) {
- for (CdcrLogPointer pointer : logPointers.values()) {
- // if we have a pointer that is not initialised, then do not remove the old tlogs
- // as we have a log reader that didn't pick them up yet.
- if (!pointer.isInitialised()) {
- return true;
- }
-
- if (pointer.tlogFile == tlog.tlogFile) {
- return true;
- }
- }
- return false;
- }
-
- @Override
- public long getLastLogId() {
- if (id != -1) return id;
- if (tlogFiles.length == 0) return -1;
- String last = tlogFiles[tlogFiles.length - 1];
- if (TLOG_NAME.length() + 1 > last.lastIndexOf('.')) {
- // old tlog created by default UpdateLog impl
- return Long.parseLong(last.substring(TLOG_NAME.length() + 1));
- } else {
- return Long.parseLong(last.substring(TLOG_NAME.length() + 1, last.lastIndexOf('.')));
- }
- }
-
- @Override
- public void add(AddUpdateCommand cmd, boolean clearCaches) {
- // Ensure we create a new tlog file following our filename format,
- // the variable tlog will be not null, and the ensureLog of the parent will be skipped
- synchronized (this) {
- if ((cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
- ensureLog(cmd.getVersion());
- }
- }
- // Then delegate to parent method
- super.add(cmd, clearCaches);
- }
-
- @Override
- public void delete(DeleteUpdateCommand cmd) {
- // Ensure we create a new tlog file following our filename format
- // the variable tlog will be not null, and the ensureLog of the parent will be skipped
- synchronized (this) {
- if ((cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
- ensureLog(cmd.getVersion());
- }
- }
- // Then delegate to parent method
- super.delete(cmd);
- }
-
- @Override
- public void deleteByQuery(DeleteUpdateCommand cmd) {
- // Ensure we create a new tlog file following our filename format
- // the variable tlog will be not null, and the ensureLog of the parent will be skipped
- synchronized (this) {
- if ((cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
- ensureLog(cmd.getVersion());
- }
- }
- // Then delegate to parent method
- super.deleteByQuery(cmd);
- }
-
- /**
- * Creates a new {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader}
- * initialised with the current list of tlogs.
- */
- @SuppressWarnings({"unchecked", "rawtypes"})
- public CdcrLogReader newLogReader() {
- return new CdcrLogReader(new ArrayList(logs), tlog);
- }
-
- /**
- * Enable the buffering of the tlogs. When buffering is activated, the update logs will not remove any
- * old transaction log files.
- */
- public void enableBuffer() {
- if (bufferToggle == null) {
- bufferToggle = this.newLogReader();
- }
- }
-
- /**
- * Disable the buffering of the tlogs.
- */
- public void disableBuffer() {
- if (bufferToggle != null) {
- bufferToggle.close();
- bufferToggle = null;
- }
- }
-
- public CdcrLogReader getBufferToggle() {
- return bufferToggle;
- }
-
- /**
- * Is the update log buffering the tlogs ?
- */
- public boolean isBuffering() {
- return bufferToggle == null ? false : true;
- }
-
- protected void ensureLog(long startVersion) {
- if (tlog == null) {
- long absoluteVersion = Math.abs(startVersion); // version is negative for deletes
- if (tlog == null) {
- String newLogName = String.format(Locale.ROOT, LOG_FILENAME_PATTERN, TLOG_NAME, id, absoluteVersion);
- tlog = new CdcrTransactionLog(new File(tlogDir, newLogName), globalStrings);
- }
-
- // push the new tlog to the opened readers
- for (CdcrLogReader reader : logPointers.keySet()) {
- reader.push(tlog);
- }
- }
- }
-
- /**
- * expert: Reset the update log before initialisation. This is called by
- * {@link org.apache.solr.handler.IndexFetcher#moveTlogFiles(File)} during a
- * a Recovery operation in order to re-initialise the UpdateLog with a new set of tlog files.
- * @see #initForRecovery(File, long)
- */
- public BufferedUpdates resetForRecovery() {
- synchronized (this) { // since we blocked updates in IndexFetcher, this synchronization shouldn't strictly be necessary.
- // If we are buffering, we need to return the related information to the index fetcher
- // for properly initialising the new update log - SOLR-8263
- BufferedUpdates bufferedUpdates = new BufferedUpdates();
- if (state == State.BUFFERING && tlog != null) {
- bufferedUpdates.tlog = tlog.tlogFile; // file to keep
- bufferedUpdates.offset = this.recoveryInfo.positionOfStart;
- }
-
- // Close readers
- for (CdcrLogReader reader : logPointers.keySet()) {
- reader.close();
- }
- logPointers.clear();
-
- // Close and clear logs
- doClose(prevTlog);
- doClose(tlog);
-
- for (TransactionLog log : logs) {
- if (log == prevTlog || log == tlog) continue;
- doClose(log);
- }
-
- logs.clear();
- newestLogsOnStartup.clear();
- tlog = prevTlog = null;
- prevMapLog = prevMapLog2 = null;
-
- map.clear();
- if (prevMap != null) prevMap.clear();
- if (prevMap2 != null) prevMap2.clear();
-
- tlogFiles = null;
- numOldRecords = 0;
-
- oldDeletes.clear();
- deleteByQueries.clear();
-
- return bufferedUpdates;
- }
- }
-
- public static class BufferedUpdates {
- public File tlog;
- public long offset;
- }
-
- /**
- * <p>
- * expert: Initialise the update log with a tlog file containing buffered updates. This is called by
- * {@link org.apache.solr.handler.IndexFetcher#moveTlogFiles(File)} during a Recovery operation.
- * This is mainly a copy of the original {@link UpdateLog#init(UpdateHandler, SolrCore)} method, but modified
- * to:
- * <ul>
- * <li>preserve the same {@link VersionInfo} instance in order to not "unblock" updates, since the
- * {@link org.apache.solr.handler.IndexFetcher#moveTlogFiles(File)} acquired a write lock from this instance.</li>
- * <li>copy the buffered updates.</li>
- * </ul>
- * @see #resetForRecovery()
- */
- public void initForRecovery(File bufferedTlog, long offset) {
- tlogFiles = getLogList(tlogDir);
- id = getLastLogId() + 1; // add 1 since we will create a new log for the next update
-
- if (debug) {
- log.debug("UpdateHandler init: tlogDir={}, existing tlogs={}, next id={}", tlogDir, Arrays.asList(tlogFiles), id);
- }
-
- TransactionLog oldLog = null;
- for (String oldLogName : tlogFiles) {
- File f = new File(tlogDir, oldLogName);
- try {
- oldLog = newTransactionLog(f, null, true);
- addOldLog(oldLog, false); // don't remove old logs on startup since more than one may be uncapped.
- } catch (Exception e) {
- SolrException.log(log, "Failure to open existing log file (non fatal) " + f, e);
- deleteFile(f);
- }
- }
-
- // Record first two logs (oldest first) at startup for potential tlog recovery.
- // It's possible that at abnormal close both "tlog" and "prevTlog" were uncapped.
- for (TransactionLog ll : logs) {
- newestLogsOnStartup.addFirst(ll);
- if (newestLogsOnStartup.size() >= 2) break;
- }
-
- // TODO: these startingVersions assume that we successfully recover from all non-complete tlogs.
- UpdateLog.RecentUpdates startingUpdates = getRecentUpdates();
- long latestVersion = startingUpdates.getMaxRecentVersion();
- try {
- startingVersions = startingUpdates.getVersions(numRecordsToKeep);
-
- // populate recent deletes list (since we can't get that info from the index)
- for (int i=startingUpdates.deleteList.size()-1; i>=0; i--) {
- DeleteUpdate du = startingUpdates.deleteList.get(i);
- oldDeletes.put(new BytesRef(du.id), new LogPtr(-1,du.version));
- }
-
- // populate recent deleteByQuery commands
- for (int i=startingUpdates.deleteByQueryList.size()-1; i>=0; i--) {
- Update update = startingUpdates.deleteByQueryList.get(i);
- @SuppressWarnings({"unchecked"})
- List<Object> dbq = (List<Object>) update.log.lookup(update.pointer);
- long version = (Long) dbq.get(1);
- String q = (String) dbq.get(2);
- trackDeleteByQuery(q, version);
- }
-
- } finally {
- startingUpdates.close();
- }
-
- // Copy buffered updates
- if (bufferedTlog != null) {
- this.copyBufferedUpdates(bufferedTlog, offset, latestVersion);
- }
- }
-
- /**
- * <p>
- * Read the entries from the given tlog file and replay them as buffered updates.
- * The buffered tlog that we are trying to copy might contain duplicate operations with the
- * current update log. During the tlog replication process, the replica might buffer update operations
- * that will be present also in the tlog files downloaded from the leader. In order to remove these
- * duplicates, it will skip any operations with a version inferior to the latest know version.
- */
- private void copyBufferedUpdates(File tlogSrc, long offsetSrc, long latestVersion) {
- recoveryInfo = new RecoveryInfo();
- state = State.BUFFERING;
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM, DistributedUpdateProcessor.DistribPhase.FROMLEADER.toString());
- SolrQueryRequest req = new LocalSolrQueryRequest(uhandler.core, params);
-
- CdcrTransactionLog src = new CdcrTransactionLog(tlogSrc, null, true);
- TransactionLog.LogReader tlogReader = src.getReader(offsetSrc);
- try {
- int operationAndFlags = 0;
- for (; ; ) {
- Object o = tlogReader.next();
- if (o == null) break; // we reached the end of the tlog
- // should currently be a List<Oper,Ver,Doc/Id>
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- operationAndFlags = (Integer) entry.get(0);
- int oper = operationAndFlags & OPERATION_MASK;
- long version = (Long) entry.get(1);
- if (version <= latestVersion) {
- // probably a buffered update that is also present in a tlog file coming from the leader,
- // skip it.
- log.debug("Dropping buffered operation - version {} < {}", version, latestVersion);
- continue;
- }
-
- switch (oper) {
- case UpdateLog.ADD: {
- SolrInputDocument sdoc = (SolrInputDocument) entry.get(entry.size() - 1);
- AddUpdateCommand cmd = new AddUpdateCommand(req);
- cmd.solrDoc = sdoc;
- cmd.setVersion(version);
- cmd.setFlags(UpdateCommand.BUFFERING);
- this.add(cmd);
- break;
- }
- case UpdateLog.DELETE: {
- byte[] idBytes = (byte[]) entry.get(2);
- DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
- cmd.setIndexedId(new BytesRef(idBytes));
- cmd.setVersion(version);
- cmd.setFlags(UpdateCommand.BUFFERING);
- this.delete(cmd);
- break;
- }
-
- case UpdateLog.DELETE_BY_QUERY: {
- String query = (String) entry.get(2);
- DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
- cmd.query = query;
- cmd.setVersion(version);
- cmd.setFlags(UpdateCommand.BUFFERING);
- this.deleteByQuery(cmd);
- break;
- }
-
- default:
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid Operation! " + oper);
- }
-
- }
- }
- catch (Exception e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to copy buffered updates", e);
- }
- finally {
- try {
- tlogReader.close();
- }
- finally {
- this.doClose(src);
- }
- }
- }
-
- private void doClose(TransactionLog theLog) {
- if (theLog != null) {
- theLog.deleteOnClose = false;
- theLog.decref();
- theLog.forceClose();
- }
- }
-
- @Override
- public void close(boolean committed, boolean deleteOnClose) {
- for (CdcrLogReader reader : new ArrayList<>(logPointers.keySet())) {
- reader.close();
- }
- super.close(committed, deleteOnClose);
- }
-
- private static class CdcrLogPointer {
-
- File tlogFile = null;
-
- private CdcrLogPointer() {
- }
-
- private void set(File tlogFile) {
- this.tlogFile = tlogFile;
- }
-
- private boolean isInitialised() {
- return tlogFile == null ? false : true;
- }
-
- @Override
- public String toString() {
- return "CdcrLogPointer(" + tlogFile + ")";
- }
-
- }
-
- public class CdcrLogReader {
-
- private TransactionLog currentTlog;
- private TransactionLog.LogReader tlogReader;
-
- // we need to use a blocking deque because of #getNumberOfRemainingRecords
- private final LinkedBlockingDeque<TransactionLog> tlogs;
- private final CdcrLogPointer pointer;
-
- /**
- * Used to record the last position of the tlog
- */
- private long lastPositionInTLog = 0;
-
- /**
- * lastVersion is used to get nextToLastVersion
- */
- private long lastVersion = -1;
-
- /**
- * nextToLastVersion is communicated by leader to replicas so that they can remove no longer needed tlogs
- * <p>
- * nextToLastVersion is used because thanks to {@link #resetToLastPosition()} lastVersion can become the current version
- */
- private long nextToLastVersion = -1;
-
- /**
- * Used to record the number of records read in the current tlog
- */
- private long numRecordsReadInCurrentTlog = 0;
-
- private CdcrLogReader(List<TransactionLog> tlogs, TransactionLog tlog) {
- this.tlogs = new LinkedBlockingDeque<>();
- this.tlogs.addAll(tlogs);
- if (tlog != null) this.tlogs.push(tlog); // ensure that the tlog being written is pushed
-
- // Register the pointer in the parent UpdateLog
- pointer = new CdcrLogPointer();
- logPointers.put(this, pointer);
-
- // If the reader is initialised while the updates log is empty, do nothing
- if ((currentTlog = this.tlogs.peekLast()) != null) {
- tlogReader = currentTlog.getReader(0);
- pointer.set(currentTlog.tlogFile);
- numRecordsReadInCurrentTlog = 0;
- log.debug("Init new tlog reader for {} - tlogReader = {}", currentTlog.tlogFile, tlogReader);
- }
- }
-
- private void push(TransactionLog tlog) {
- this.tlogs.push(tlog);
-
- // The reader was initialised while the update logs was empty, or reader was exhausted previously,
- // we have to update the current tlog and the associated tlog reader.
- if (currentTlog == null && !tlogs.isEmpty()) {
- currentTlog = tlogs.peekLast();
- tlogReader = currentTlog.getReader(0);
- pointer.set(currentTlog.tlogFile);
- numRecordsReadInCurrentTlog = 0;
- log.debug("Init new tlog reader for {} - tlogReader = {}", currentTlog.tlogFile, tlogReader);
- }
- }
-
- /**
- * Expert: Instantiate a sub-reader. A sub-reader is used for batch updates. It allows to iterates over the
- * update logs entries without modifying the state of the parent log reader. If the batch update fails, the state
- * of the sub-reader is discarded and the state of the parent reader is not modified. If the batch update
- * is successful, the sub-reader is used to fast forward the parent reader with the method
- * {@link #forwardSeek(org.apache.solr.update.CdcrUpdateLog.CdcrLogReader)}.
- */
- public CdcrLogReader getSubReader() {
- // Add the last element of the queue to properly initialise the pointer and log reader
- CdcrLogReader clone = new CdcrLogReader(new ArrayList<TransactionLog>(), this.tlogs.peekLast());
- clone.tlogs.clear(); // clear queue before copy
- clone.tlogs.addAll(tlogs); // perform a copy of the list
- clone.lastPositionInTLog = this.lastPositionInTLog;
- clone.numRecordsReadInCurrentTlog = this.numRecordsReadInCurrentTlog;
- clone.lastVersion = this.lastVersion;
- clone.nextToLastVersion = this.nextToLastVersion;
-
- // If the update log is not empty, we need to initialise the tlog reader
- // NB: the tlogReader is equal to null if the update log is empty
- if (tlogReader != null) {
- clone.tlogReader.close();
- clone.tlogReader = currentTlog.getReader(this.tlogReader.currentPos());
- }
-
- return clone;
- }
-
- /**
- * Expert: Fast forward this log reader with a log subreader. The subreader will be closed after calling this
- * method. In order to avoid unexpected results, the log
- * subreader must be created from this reader with the method {@link #getSubReader()}.
- */
- public void forwardSeek(CdcrLogReader subReader) {
- // If a subreader has a null tlog reader, does nothing
- // This can happened if a subreader is instantiated from a non-initialised parent reader, or if the subreader
- // has been closed.
- if (subReader.tlogReader == null) {
- return;
- }
-
- tlogReader.close(); // close the existing reader, a new one will be created
- while (this.tlogs.peekLast().id < subReader.tlogs.peekLast().id) {
- tlogs.removeLast();
- currentTlog = tlogs.peekLast();
- }
- assert this.tlogs.peekLast().id == subReader.tlogs.peekLast().id : this.tlogs.peekLast().id+" != "+subReader.tlogs.peekLast().id;
- this.pointer.set(currentTlog.tlogFile);
- this.lastPositionInTLog = subReader.lastPositionInTLog;
- this.numRecordsReadInCurrentTlog = subReader.numRecordsReadInCurrentTlog;
- this.lastVersion = subReader.lastVersion;
- this.nextToLastVersion = subReader.nextToLastVersion;
- this.tlogReader = currentTlog.getReader(subReader.tlogReader.currentPos());
- }
-
- /**
- * Advances to the next log entry in the updates log and returns the log entry itself.
- * Returns null if there are no more log entries in the updates log.<br>
- * <p>
- * <b>NOTE:</b> after the reader has exhausted, you can call again this method since the updates
- * log might have been updated with new entries.
- */
- public Object next() throws IOException, InterruptedException {
- while (!tlogs.isEmpty()) {
- lastPositionInTLog = tlogReader.currentPos();
- Object o = tlogReader.next();
-
- if (o != null) {
- pointer.set(currentTlog.tlogFile);
- nextToLastVersion = lastVersion;
- lastVersion = getVersion(o);
- numRecordsReadInCurrentTlog++;
- return o;
- }
-
- if (tlogs.size() > 1) { // if the current tlog is not the newest one, we can advance to the next one
- tlogReader.close();
- tlogs.removeLast();
- currentTlog = tlogs.peekLast();
- tlogReader = currentTlog.getReader(0);
- pointer.set(currentTlog.tlogFile);
- numRecordsReadInCurrentTlog = 0;
- log.debug("Init new tlog reader for {} - tlogReader = {}", currentTlog.tlogFile, tlogReader);
- } else {
- // the only tlog left is the new tlog which is currently being written,
- // we should not remove it as we have to try to read it again later.
- return null;
- }
- }
-
- return null;
- }
-
- /**
- * Advances to the first beyond the current whose version number is greater
- * than or equal to <i>targetVersion</i>.<br>
- * Returns true if the reader has been advanced. If <i>targetVersion</i> is
- * greater than the highest version number in the updates log, the reader
- * has been advanced to the end of the current tlog, and a call to
- * {@link #next()} will probably return null.<br>
- * Returns false if <i>targetVersion</i> is lower than the oldest known entry.
- * In this scenario, it probably means that there is a gap in the updates log.<br>
- * <p>
- * <b>NOTE:</b> This method must be called before the first call to {@link #next()}.
- */
- public boolean seek(long targetVersion) throws IOException, InterruptedException {
- Object o;
- // version is negative for deletes - ensure that we are manipulating absolute version numbers.
- targetVersion = Math.abs(targetVersion);
-
- if (tlogs.isEmpty() || !this.seekTLog(targetVersion)) {
- return false;
- }
-
- // now that we might be on the right tlog, iterates over the entries to find the one we are looking for
- while ((o = this.next()) != null) {
- if (this.getVersion(o) >= targetVersion) {
- this.resetToLastPosition();
- return true;
- }
- }
-
- return true;
- }
-
- /**
- * Seeks the tlog associated to the target version by using the updates log index,
- * and initialises the log reader to the start of the tlog. Returns true if it was able
- * to seek the corresponding tlog, false if the <i>targetVersion</i> is lower than the
- * oldest known entry (which probably indicates a gap).<br>
- * <p>
- * <b>NOTE:</b> This method might modify the tlog queue by removing tlogs that are older
- * than the target version.
- */
- private boolean seekTLog(long targetVersion) {
- // if the target version is lower than the oldest known entry, we have probably a gap.
- if (targetVersion < ((CdcrTransactionLog) tlogs.peekLast()).startVersion) {
- return false;
- }
-
- // closes existing reader before performing seek and possibly modifying the queue;
- tlogReader.close();
-
- // iterates over the queue and removes old tlogs
- TransactionLog last = null;
- while (tlogs.size() > 1) {
- if (((CdcrTransactionLog) tlogs.peekLast()).startVersion >= targetVersion) {
- break;
- }
- last = tlogs.pollLast();
- }
-
- // the last tlog removed is the one we look for, add it back to the queue
- if (last != null) tlogs.addLast(last);
-
- currentTlog = tlogs.peekLast();
- tlogReader = currentTlog.getReader(0);
- pointer.set(currentTlog.tlogFile);
- numRecordsReadInCurrentTlog = 0;
-
- return true;
- }
-
- /**
- * Extracts the version number and converts it to its absolute form.
- */
- private long getVersion(Object o) {
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- // version is negative for delete, ensure that we are manipulating absolute version numbers
- return Math.abs((Long) entry.get(1));
- }
-
- /**
- * If called after {@link #next()}, it resets the reader to its last position.
- */
- public void resetToLastPosition() {
- try {
- if (tlogReader != null) {
- tlogReader.fis.seek(lastPositionInTLog);
- numRecordsReadInCurrentTlog--;
- lastVersion = nextToLastVersion;
- }
- } catch (IOException e) {
- log.error("Failed to seek last position in tlog", e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to seek last position in tlog", e);
- }
- }
-
- /**
- * Returns the number of remaining records (including commit but excluding header) to be read in the logs.
- */
- public long getNumberOfRemainingRecords() {
- long numRemainingRecords = 0;
-
- synchronized (tlogs) {
- for (TransactionLog tlog : tlogs) {
- numRemainingRecords += tlog.numRecords() - 1; // minus 1 as the number of records returned by the tlog includes the header
- }
- }
-
- return numRemainingRecords - numRecordsReadInCurrentTlog;
- }
-
- /**
- * Closes streams and remove the associated {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogPointer} from the
- * parent {@link org.apache.solr.update.CdcrUpdateLog}.
- */
- public void close() {
- if (tlogReader != null) {
- tlogReader.close();
- tlogReader = null;
- currentTlog = null;
- }
- tlogs.clear();
- logPointers.remove(this);
- }
-
- /**
- * Returns the absolute form of the version number of the last entry read. If the current version is equal
- * to 0 (because of a commit), it will return the next to last version number.
- */
- public long getLastVersion() {
- return lastVersion == 0 ? nextToLastVersion : lastVersion;
- }
- }
-
-}
-
diff --git a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index 53dcb3e..2454dc4 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -18,12 +18,10 @@ package org.apache.solr.update;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
-import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -79,14 +77,6 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
protected final ReentrantLock commitLock = new ReentrantLock();
-
- private AtomicBoolean cdcrRunning = new AtomicBoolean();
-
- private volatile Future<Boolean> cdcrBootstrapFuture;
-
- @SuppressWarnings({"rawtypes"})
- private volatile Callable cdcrBootstrapCallable;
-
@Deprecated
public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
this(directoryFactory, new RecoveryStrategy.Builder());
@@ -427,35 +417,4 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
public Lock getRecoveryLock() {
return recoveryLock;
}
-
- @Override
- public boolean getCdcrBootstrapRunning() {
- return cdcrRunning.get();
- }
-
- @Override
- public void setCdcrBootstrapRunning(boolean cdcrRunning) {
- this.cdcrRunning.set(cdcrRunning);
- }
-
- @Override
- public Future<Boolean> getCdcrBootstrapFuture() {
- return cdcrBootstrapFuture;
- }
-
- @Override
- public void setCdcrBootstrapFuture(Future<Boolean> cdcrBootstrapFuture) {
- this.cdcrBootstrapFuture = cdcrBootstrapFuture;
- }
-
- @Override
- @SuppressWarnings({"rawtypes"})
- public Callable getCdcrBootstrapCallable() {
- return cdcrBootstrapCallable;
- }
-
- @Override
- public void setCdcrBootstrapCallable(@SuppressWarnings({"rawtypes"})Callable cdcrBootstrapCallable) {
- this.cdcrBootstrapCallable = cdcrBootstrapCallable;
- }
}
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
index eddd5b7..c8f61d5 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
@@ -18,8 +18,6 @@ package org.apache.solr.update;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
import java.util.concurrent.locks.Lock;
import org.apache.lucene.index.IndexWriter;
@@ -186,21 +184,6 @@ public abstract class SolrCoreState {
public abstract Lock getRecoveryLock();
- // These are needed to properly synchronize the bootstrapping when the
- // in the target DC require a full sync.
- public abstract boolean getCdcrBootstrapRunning();
-
- public abstract void setCdcrBootstrapRunning(boolean cdcrRunning);
-
- public abstract Future<Boolean> getCdcrBootstrapFuture();
-
- public abstract void setCdcrBootstrapFuture(Future<Boolean> cdcrBootstrapFuture);
-
- @SuppressWarnings("rawtypes")
- public abstract Callable getCdcrBootstrapCallable();
-
- public abstract void setCdcrBootstrapCallable(@SuppressWarnings("rawtypes") Callable cdcrBootstrapCallable);
-
public Throwable getTragicException() throws IOException {
RefCounted<IndexWriter> ref = getIndexWriter(null);
if (ref == null) return null;
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 85ee40a..b928878 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1517,8 +1517,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
update.version = version;
if (oper == UpdateLog.UPDATE_INPLACE) {
- if ((update.log instanceof CdcrTransactionLog && entry.size() == 6) ||
- (!(update.log instanceof CdcrTransactionLog) && entry.size() == 5)) {
+ if (entry.size() == 5) {
update.previousVersion = (Long) entry.get(UpdateLog.PREV_VERSION_IDX);
}
}
diff --git a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java
deleted file mode 100644
index 180784a..0000000
--- a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.update.processor;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.DeleteUpdateCommand;
-import org.apache.solr.update.UpdateCommand;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * Extends {@link org.apache.solr.update.processor.DistributedUpdateProcessor} to force peer sync logic
- * for every updates. This ensures that the version parameter sent by the source cluster is kept
- * by the target cluster.
- * </p>
- * @deprecated since 8.6
- */
-@Deprecated(since = "8.6")
-public class CdcrUpdateProcessor extends DistributedZkUpdateProcessor {
-
- public static final String CDCR_UPDATE = "cdcr.update";
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public CdcrUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
- super(req, rsp, next);
- }
-
- @Override
- protected boolean versionAdd(AddUpdateCommand cmd) throws IOException {
- /*
- temporarily set the PEER_SYNC flag so that DistributedUpdateProcessor.versionAdd doesn't execute leader logic
- but the else part of that if. That way version remains preserved.
-
- we cannot set the flag for the whole processAdd method because DistributedUpdateProcessor.setupRequest() would set
- isLeader to false which wouldn't work
- */
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() | UpdateCommand.PEER_SYNC); // we need super.versionAdd() to set leaderLogic to false
- }
-
- boolean result = super.versionAdd(cmd);
-
- // unset the flag to avoid unintended consequences down the chain
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() & ~UpdateCommand.PEER_SYNC);
- }
-
- return result;
- }
-
- @Override
- protected boolean versionDelete(DeleteUpdateCommand cmd) throws IOException {
- /*
- temporarily set the PEER_SYNC flag so that DistributedUpdateProcessor.deleteAdd doesn't execute leader logic
- but the else part of that if. That way version remains preserved.
-
- we cannot set the flag for the whole processDelete method because DistributedUpdateProcessor.setupRequest() would set
- isLeader to false which wouldn't work
- */
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() | UpdateCommand.PEER_SYNC); // we need super.versionAdd() to set leaderLogic to false
- }
-
- boolean result = super.versionDelete(cmd);
-
- // unset the flag to avoid unintended consequences down the chain
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() & ~UpdateCommand.PEER_SYNC);
- }
-
- return result;
- }
-
- protected ModifiableSolrParams filterParams(SolrParams params) {
- ModifiableSolrParams result = super.filterParams(params);
- if (params.get(CDCR_UPDATE) != null) {
- result.set(CDCR_UPDATE, "");
-// if (params.get(DistributedUpdateProcessor.VERSION_FIELD) == null) {
-// log.warn("+++ cdcr.update but no version field, params are: " + params);
-// } else {
-// log.info("+++ cdcr.update version present, params are: " + params);
-// }
- result.set(CommonParams.VERSION_FIELD, params.get(CommonParams.VERSION_FIELD));
- }
-
- return result;
- }
-
- @Override
- protected void versionDeleteByQuery(DeleteUpdateCommand cmd) throws IOException {
- /*
- temporarily set the PEER_SYNC flag so that DistributedUpdateProcessor.versionDeleteByQuery doesn't execute leader logic
- That way version remains preserved.
-
- */
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() | UpdateCommand.PEER_SYNC); // we need super.versionDeleteByQuery() to set leaderLogic to false
- }
-
- super.versionDeleteByQuery(cmd);
-
- // unset the flag to avoid unintended consequences down the chain
- if (cmd.getReq().getParams().get(CDCR_UPDATE) != null) {
- cmd.setFlags(cmd.getFlags() & ~UpdateCommand.PEER_SYNC);
- }
- }
-}
-
diff --git a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessorFactory.java
deleted file mode 100644
index 08cec4f..0000000
--- a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessorFactory.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.update.processor;
-
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-
-/**
- * Factory for {@link org.apache.solr.update.processor.CdcrUpdateProcessor}.
- *
- * @see org.apache.solr.update.processor.CdcrUpdateProcessor
- * @since 6.0.0
- */
-public class CdcrUpdateProcessorFactory
- extends UpdateRequestProcessorFactory
- implements DistributingUpdateProcessorFactory {
-
- @Override
- public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
-
- }
-
- @Override
- public CdcrUpdateProcessor getInstance(SolrQueryRequest req,
- SolrQueryResponse rsp, UpdateRequestProcessor next) {
-
- return new CdcrUpdateProcessor(req, rsp, next);
- }
-
-}
-
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcr.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcr.xml
deleted file mode 100644
index c6b360c..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcr.xml
+++ /dev/null
@@ -1,77 +0,0 @@
-<?xml version="1.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.
--->
-
-<config>
- <jmx/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
- <!-- used to keep RAM reqs down for HdfsDirectoryFactory -->
- <bool name="solr.hdfs.blockcache.enabled">${solr.hdfs.blockcache.enabled:true}</bool>
- <int name="solr.hdfs.blockcache.blocksperbank">${solr.hdfs.blockcache.blocksperbank:1024}</int>
- <str name="solr.hdfs.home">${solr.hdfs.home:}</str>
- <str name="solr.hdfs.confdir">${solr.hdfs.confdir:}</str>
- <str name="solr.hdfs.blockcache.global">${solr.hdfs.blockcache.global:false}</str>
- </directoryFactory>
-
- <schemaFactory class="ClassicIndexSchemaFactory"/>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
-
- <requestHandler name="/select" class="solr.SearchHandler">
- </requestHandler>
-
- <requestHandler name="/update" class="solr.UpdateRequestHandler">
- <lst name="defaults">
- <str name="update.chain">cdcr-processor-chain</str>
- </lst>
- </requestHandler>
-
- <updateRequestProcessorChain name="cdcr-processor-chain">
- <processor class="solr.CdcrUpdateProcessorFactory"/>
- <processor class="solr.RunUpdateProcessorFactory"/>
- </updateRequestProcessorChain>
-
- <requestHandler name="/cdcr" class="solr.CdcrRequestHandler">
- <lst name="replica">
- <str name="zkHost">${zkHost}</str>
- <str name="source">source_collection</str>
- <str name="target">target_collection</str>
- </lst>
- <lst name="replicator">
- <str name="threadPoolSize">8</str>
- <str name="schedule">1000</str>
- <str name="batchSize">64</str>
- </lst>
- <lst name="updateLogSynchronizer">
- <str name="schedule">1000</str>
- </lst>
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
-</config>
-
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcrupdatelog.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcrupdatelog.xml
deleted file mode 100644
index 86b2d2b..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cdcrupdatelog.xml
+++ /dev/null
@@ -1,49 +0,0 @@
-<?xml version="1.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.
--->
-
-<config>
- <jmx/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
- <!-- used to keep RAM reqs down for HdfsDirectoryFactory -->
- <bool name="solr.hdfs.blockcache.enabled">${solr.hdfs.blockcache.enabled:true}</bool>
- <int name="solr.hdfs.blockcache.blocksperbank">${solr.hdfs.blockcache.blocksperbank:1024}</int>
- <str name="solr.hdfs.home">${solr.hdfs.home:}</str>
- <str name="solr.hdfs.confdir">${solr.hdfs.confdir:}</str>
- <str name="solr.hdfs.blockcache.global">${solr.hdfs.blockcache.global:false}</str>
- </directoryFactory>
-
- <schemaFactory class="ClassicIndexSchemaFactory"/>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
-
- <requestHandler name="/select" class="solr.SearchHandler">
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
-</config>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/managed-schema b/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/managed-schema
deleted file mode 100644
index 2df6c0a..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/managed-schema
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-<schema name="minimal" version="1.1">
- <types>
- <fieldType name="string" class="solr.StrField"/>
- <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
- </types>
- <fields>
- <field name="id" type="string" indexed="true" stored="true"/>
- <field name="_version_" type="long" indexed="true" stored="true"/>
- <dynamicField name="*" type="string" indexed="true" stored="true"/>
- </fields>
- <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/solrconfig.xml
deleted file mode 100644
index da548c4..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-cluster1/conf/solrconfig.xml
+++ /dev/null
@@ -1,80 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- This is a "kitchen sink" config file that tests can use.
- When writting a new test, feel free to add *new* items (plugins,
- config options, etc...) as long as they don't break any existing
- tests. if you need to test something esoteric please add a new
- "solrconfig-your-esoteric-purpose.xml" config file.
-
- Note in particular that this test is used by MinimalSchemaTest so
- Anything added to this file needs to work correctly even if there
- is now uniqueKey or defaultSearch Field.
- -->
-
-<config>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <directoryFactory name="DirectoryFactory"
- class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <updateRequestProcessorChain name="cdcr-processor-chain">
- <processor class="solr.CdcrUpdateProcessorFactory"/>
- <processor class="solr.RunUpdateProcessorFactory"/>
- </updateRequestProcessorChain>
-
- <requestHandler name="/cdcr" class="solr.CdcrRequestHandler">
- <lst name="replica">
- <str name="zkHost">${cdcr.cluster2.zkHost}</str>
- <str name="source">cdcr-cluster1</str>
- <str name="target">cdcr-cluster2</str>
- </lst>
- <lst name="replicator">
- <str name="threadPoolSize">1</str>
- <str name="schedule">1000</str>
- <str name="batchSize">1000</str>
- </lst>
- <lst name="updateLogSynchronizer">
- <str name="schedule">1000</str>
- </lst>
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
- <requestHandler name="/select" class="solr.SearchHandler" />
-
- <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell,/browse">
- <lst name="defaults">
- <str name="df">_text_</str>
- </lst>
- </initParams>
-
- <requestHandler name="/update" class="solr.UpdateRequestHandler">
- <lst name="defaults">
- <str name="update.chain">cdcr-processor-chain</str>
- </lst>
- </requestHandler>
-</config>
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/managed-schema b/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/managed-schema
deleted file mode 100644
index 2df6c0a..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/managed-schema
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-<schema name="minimal" version="1.1">
- <types>
- <fieldType name="string" class="solr.StrField"/>
- <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
- </types>
- <fields>
- <field name="id" type="string" indexed="true" stored="true"/>
- <field name="_version_" type="long" indexed="true" stored="true"/>
- <dynamicField name="*" type="string" indexed="true" stored="true"/>
- </fields>
- <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/solrconfig.xml
deleted file mode 100644
index 8e26d45..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-cluster2/conf/solrconfig.xml
+++ /dev/null
@@ -1,80 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- This is a "kitchen sink" config file that tests can use.
- When writting a new test, feel free to add *new* items (plugins,
- config options, etc...) as long as they don't break any existing
- tests. if you need to test something esoteric please add a new
- "solrconfig-your-esoteric-purpose.xml" config file.
-
- Note in particular that this test is used by MinimalSchemaTest so
- Anything added to this file needs to work correctly even if there
- is now uniqueKey or defaultSearch Field.
- -->
-
-<config>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <directoryFactory name="DirectoryFactory"
- class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <updateRequestProcessorChain name="cdcr-processor-chain">
- <processor class="solr.CdcrUpdateProcessorFactory"/>
- <processor class="solr.RunUpdateProcessorFactory"/>
- </updateRequestProcessorChain>
-
- <requestHandler name="/cdcr" class="solr.CdcrRequestHandler">
- <lst name="replica">
- <str name="zkHost">${cdcr.cluster1.zkHost}</str>
- <str name="source">cdcr-cluster2</str>
- <str name="target">cdcr-cluster1</str>
- </lst>
- <lst name="replicator">
- <str name="threadPoolSize">1</str>
- <str name="schedule">1000</str>
- <str name="batchSize">1000</str>
- </lst>
- <lst name="updateLogSynchronizer">
- <str name="schedule">1000</str>
- </lst>
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
- <requestHandler name="/select" class="solr.SearchHandler" />
-
- <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell,/browse">
- <lst name="defaults">
- <str name="df">_text_</str>
- </lst>
- </initParams>
-
- <requestHandler name="/update" class="solr.UpdateRequestHandler">
- <lst name="defaults">
- <str name="update.chain">cdcr-processor-chain</str>
- </lst>
- </requestHandler>
-</config>
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/schema.xml
deleted file mode 100644
index 2df6c0a..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/schema.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-<schema name="minimal" version="1.1">
- <types>
- <fieldType name="string" class="solr.StrField"/>
- <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
- </types>
- <fields>
- <field name="id" type="string" indexed="true" stored="true"/>
- <field name="_version_" type="long" indexed="true" stored="true"/>
- <dynamicField name="*" type="string" indexed="true" stored="true"/>
- </fields>
- <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/solrconfig.xml
deleted file mode 100644
index e63d9a6..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-source-disabled/conf/solrconfig.xml
+++ /dev/null
@@ -1,60 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- This is a "kitchen sink" config file that tests can use.
- When writting a new test, feel free to add *new* items (plugins,
- config options, etc...) as long as they don't break any existing
- tests. if you need to test something esoteric please add a new
- "solrconfig-your-esoteric-purpose.xml" config file.
-
- Note in particular that this test is used by MinimalSchemaTest so
- Anything added to this file needs to work correctly even if there
- is now uniqueKey or defaultSearch Field.
- -->
-
-<config>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <directoryFactory name="DirectoryFactory"
- class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
- <schemaFactory class="ClassicIndexSchemaFactory"/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <commitWithin>
- <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
- </commitWithin>
-
- <updateLog>
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
-
- </updateHandler>
- <requestHandler name="/select" class="solr.SearchHandler">
- <lst name="defaults">
- <str name="echoParams">explicit</str>
- <str name="indent">true</str>
- <str name="df">text</str>
- </lst>
-
- </requestHandler>
-</config>
-
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-source/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cdcr-source/conf/schema.xml
deleted file mode 100644
index 2df6c0a..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-source/conf/schema.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-<schema name="minimal" version="1.1">
- <types>
- <fieldType name="string" class="solr.StrField"/>
- <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
- </types>
- <fields>
- <field name="id" type="string" indexed="true" stored="true"/>
- <field name="_version_" type="long" indexed="true" stored="true"/>
- <dynamicField name="*" type="string" indexed="true" stored="true"/>
- </fields>
- <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-source/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cdcr-source/conf/solrconfig.xml
deleted file mode 100644
index 6469038..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-source/conf/solrconfig.xml
+++ /dev/null
@@ -1,75 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- This is a "kitchen sink" config file that tests can use.
- When writting a new test, feel free to add *new* items (plugins,
- config options, etc...) as long as they don't break any existing
- tests. if you need to test something esoteric please add a new
- "solrconfig-your-esoteric-purpose.xml" config file.
-
- Note in particular that this test is used by MinimalSchemaTest so
- Anything added to this file needs to work correctly even if there
- is now uniqueKey or defaultSearch Field.
- -->
-
-<config>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <directoryFactory name="DirectoryFactory"
- class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <updateRequestProcessorChain name="cdcr-processor-chain">
- <processor class="solr.CdcrUpdateProcessorFactory"/>
- <processor class="solr.RunUpdateProcessorFactory"/>
- </updateRequestProcessorChain>
-
- <requestHandler name="/cdcr" class="solr.CdcrRequestHandler">
- <lst name="replica">
- <str name="zkHost">${cdcr.target.zkHost}</str>
- <str name="source">cdcr-source</str>
- <str name="target">cdcr-target</str>
- </lst>
- <lst name="replicator">
- <str name="threadPoolSize">1</str>
- <str name="schedule">1000</str>
- <str name="batchSize">1000</str>
- </lst>
- <lst name="updateLogSynchronizer">
- <str name="schedule">1000</str>
- </lst>
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
- <requestHandler name="/select" class="solr.SearchHandler" />
-
- <requestHandler name="/update" class="solr.UpdateRequestHandler">
- <lst name="defaults">
- <str name="update.chain">cdcr-processor-chain</str>
- </lst>
- </requestHandler>
-</config>
-
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-target/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cdcr-target/conf/schema.xml
deleted file mode 100644
index 2df6c0a..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-target/conf/schema.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-<schema name="minimal" version="1.1">
- <types>
- <fieldType name="string" class="solr.StrField"/>
- <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
- </types>
- <fields>
- <field name="id" type="string" indexed="true" stored="true"/>
- <field name="_version_" type="long" indexed="true" stored="true"/>
- <dynamicField name="*" type="string" indexed="true" stored="true"/>
- </fields>
- <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cdcr-target/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cdcr-target/conf/solrconfig.xml
deleted file mode 100644
index bb4a774..0000000
--- a/solr/core/src/test-files/solr/configsets/cdcr-target/conf/solrconfig.xml
+++ /dev/null
@@ -1,62 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- This is a "kitchen sink" config file that tests can use.
- When writting a new test, feel free to add *new* items (plugins,
- config options, etc...) as long as they don't break any existing
- tests. if you need to test something esoteric please add a new
- "solrconfig-your-esoteric-purpose.xml" config file.
-
- Note in particular that this test is used by MinimalSchemaTest so
- Anything added to this file needs to work correctly even if there
- is now uniqueKey or defaultSearch Field.
- -->
-
-<config>
-
- <dataDir>${solr.data.dir:}</dataDir>
-
- <directoryFactory name="DirectoryFactory"
- class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
-
- <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
- <updateRequestProcessorChain name="cdcr-processor-chain">
- <processor class="solr.CdcrUpdateProcessorFactory"/>
- <processor class="solr.RunUpdateProcessorFactory"/>
- </updateRequestProcessorChain>
-
- <requestHandler name="/cdcr" class="solr.CdcrRequestHandler">
- </requestHandler>
-
- <updateHandler class="solr.DirectUpdateHandler2">
- <updateLog class="solr.CdcrUpdateLog">
- <str name="dir">${solr.ulog.dir:}</str>
- </updateLog>
- </updateHandler>
-
- <requestHandler name="/select" class="solr.SearchHandler" />
-
- <requestHandler name="/update" class="solr.UpdateRequestHandler">
- <lst name="defaults">
- <str name="update.chain">cdcr-processor-chain</str>
- </lst>
- </requestHandler>
-</config>
-
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
deleted file mode 100644
index aa35de1..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
+++ /dev/null
@@ -1,906 +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.cdcr;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.cloud.AbstractDistribZkTestBase;
-import org.apache.solr.cloud.AbstractZkTestCase;
-import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.handler.CdcrParams;
-import org.apache.solr.util.TimeOut;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
-import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.common.cloud.ZkStateReader.CLUSTER_PROPS;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.handler.admin.CoreAdminHandler.COMPLETED;
-import static org.apache.solr.handler.admin.CoreAdminHandler.RESPONSE_STATUS;
-
-/**
- * <p>
- * Abstract class for CDCR unit testing. This class emulates two clusters, a source and target, by using different
- * collections in the same SolrCloud cluster. Therefore, the two clusters will share the same Zookeeper cluster. In
- * real scenario, the two collections/clusters will likely have their own zookeeper cluster.
- * </p>
- * <p>
- * This class will automatically create two collections, the source and the target. Each collection will have
- * {@link #shardCount} shards, and {@link #replicationFactor} replicas per shard. One jetty instance will
- * be created per core.
- * </p>
- * <p>
- * The source and target collection can be reinitialised at will by calling {@link #clearSourceCollection()} and
- * {@link #clearTargetCollection()}. After reinitialisation, a collection will have a new fresh index and update log.
- * </p>
- * <p>
- * Servers can be restarted at will by calling
- * {@link #restartServer(BaseCdcrDistributedZkTest.CloudJettyRunner)} or
- * {@link #restartServers(java.util.List)}.
- * </p>
- * <p>
- * The creation of the target collection can be disabled with the flag {@link #createTargetCollection};
- * </p>
- * <p>
- * NB: We cannot use multiple cores per jetty instance, as jetty will load only one core when restarting. It seems
- * that this is a limitation of the {@link org.apache.solr.client.solrj.embedded.JettySolrRunner}. This class
- * tries to ensure that there always is one single core per jetty instance.
- * </p>
- */
-public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- protected int shardCount = 2;
- protected int replicationFactor = 2;
- protected boolean createTargetCollection = true;
-
- private static final String CDCR_PATH = "/cdcr";
-
- protected static final String SOURCE_COLLECTION = "source_collection";
- protected static final String TARGET_COLLECTION = "target_collection";
-
- public static final String SHARD1 = "shard1";
- public static final String SHARD2 = "shard2";
-
- @Override
- protected String getCloudSolrConfig() {
- return "solrconfig-cdcr.xml";
- }
-
- @Override
- public void distribSetUp() throws Exception {
- super.distribSetUp();
-
- if (shardCount > 0) {
- System.setProperty("numShards", Integer.toString(shardCount));
- } else {
- System.clearProperty("numShards");
- }
-
- if (isSSLMode()) {
- System.clearProperty("urlScheme");
- ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
- AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
- try {
- zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
- Utils.toJSON(Collections.singletonMap("urlScheme", "https")),
- CreateMode.PERSISTENT, true);
- } catch (KeeperException.NodeExistsException e) {
- ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(ZkStateReader.CLUSTER_PROPS,
- null, null, true));
- props = props.plus("urlScheme", "https");
- zkStateReader.getZkClient().setData(CLUSTER_PROPS, Utils.toJSON(props), true);
- } finally {
- zkStateReader.close();
- }
- }
- }
-
- @Override
- protected void createServers(int numServers) throws Exception {
- }
-
- @BeforeClass
- public static void beforeClass() {
- System.setProperty("solrcloud.update.delay", "0");
- }
-
- @AfterClass
- public static void afterClass() throws Exception {
- System.clearProperty("solrcloud.update.delay");
- }
-
- @Before
- @SuppressWarnings({"rawtypes"})
- public void baseBefore() throws Exception {
- this.createSourceCollection();
- if (this.createTargetCollection) this.createTargetCollection();
- RandVal.uniqueValues = new HashSet(); //reset random values
- }
-
- @After
- public void baseAfter() throws Exception {
- for (List<CloudJettyRunner> runners : cloudJettys.values()) {
- for (CloudJettyRunner runner : runners) {
- runner.client.close();
- }
- }
- destroyServers();
- }
-
- protected CloudSolrClient createCloudClient(String defaultCollection) {
- CloudSolrClient server = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean());
- if (defaultCollection != null) server.setDefaultCollection(defaultCollection);
- return server;
- }
-
- protected SolrInputDocument getDoc(Object... fields) throws Exception {
- SolrInputDocument doc = new SolrInputDocument();
- addFields(doc, fields);
- return doc;
- }
-
- protected void index(String collection, SolrInputDocument doc) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- client.add(doc);
- client.commit(true, true);
- } finally {
- client.close();
- }
- }
-
- protected void index(String collection, List<SolrInputDocument> docs) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- client.add(docs);
- client.commit(true, true);
- } finally {
- client.close();
- }
- }
-
- protected void deleteById(String collection, List<String> ids) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- client.deleteById(ids);
- client.commit(true, true);
- } finally {
- client.close();
- }
- }
-
- protected void deleteByQuery(String collection, String q) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- client.deleteByQuery(q);
- client.commit(true, true);
- } finally {
- client.close();
- }
- }
-
- /**
- * Invokes a commit on the given collection.
- */
- protected void commit(String collection) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- client.commit(true, true);
- } finally {
- client.close();
- }
- }
-
- /**
- * Assert the number of documents in a given collection
- */
- protected void assertNumDocs(int expectedNumDocs, String collection)
- throws SolrServerException, IOException, InterruptedException {
- CloudSolrClient client = createCloudClient(collection);
- try {
- int cnt = 30; // timeout after 15 seconds
- AssertionError lastAssertionError = null;
- while (cnt > 0) {
- try {
- assertEquals(expectedNumDocs, client.query(new SolrQuery("*:*")).getResults().getNumFound());
- return;
- }
- catch (AssertionError e) {
- lastAssertionError = e;
- cnt--;
- Thread.sleep(500);
- }
- }
- throw new AssertionError("Timeout while trying to assert number of documents @ " + collection, lastAssertionError);
- } finally {
- client.close();
- }
- }
-
- /**
- * Invokes a CDCR action on a given node.
- */
- @SuppressWarnings({"rawtypes"})
- protected NamedList invokeCdcrAction(CloudJettyRunner jetty, CdcrParams.CdcrAction action) throws Exception {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, action.toString());
-
- SolrRequest request = new QueryRequest(params);
- request.setPath(CDCR_PATH);
-
- return jetty.client.request(request);
- }
-
- protected void waitForCdcrStateReplication(String collection) throws Exception {
- log.info("Wait for CDCR state to replicate - collection: {}", collection);
-
- int cnt = 30;
- while (cnt > 0) {
- @SuppressWarnings({"rawtypes"})
- NamedList status = null;
- boolean allEquals = true;
- for (CloudJettyRunner jetty : cloudJettys.get(collection)) { // check all replicas
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = invokeCdcrAction(jetty, CdcrParams.CdcrAction.STATUS);
- if (status == null) {
- status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- continue;
- }
- allEquals &= status.equals(rsp.get(CdcrParams.CdcrAction.STATUS.toLower()));
- }
-
- if (allEquals) {
- break;
- }
- else {
- if (cnt == 0) {
- throw new RuntimeException("Timeout waiting for CDCR state to replicate: collection="+collection);
- }
- cnt--;
- Thread.sleep(500);
- }
- }
-
- log.info("CDCR state is identical across nodes - collection: {}", collection);
- }
-
- /**
- * Assert the state of CDCR on each nodes of the given collection.
- */
- protected void assertState(String collection, CdcrParams.ProcessState processState, CdcrParams.BufferState bufferState)
- throws Exception {
- this.waitForCdcrStateReplication(collection); // ensure that cdcr state is replicated and stable
- for (CloudJettyRunner jetty : cloudJettys.get(collection)) { // check all replicas
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = invokeCdcrAction(jetty, CdcrParams.CdcrAction.STATUS);
- @SuppressWarnings({"rawtypes"})
- NamedList status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- assertEquals(processState.toLower(), status.get(CdcrParams.ProcessState.getParam()));
- assertEquals(bufferState.toLower(), status.get(CdcrParams.BufferState.getParam()));
- }
- }
-
- /**
- * A mapping between collection and node names. This is used when creating the collection in
- * {@link #createCollection(String)}.
- */
- private Map<String, List<String>> collectionToNodeNames = new HashMap<>();
-
- /**
- * Starts the servers, saves and associates the node names to the source collection,
- * and finally creates the source collection.
- */
- private void createSourceCollection() throws Exception {
- List<String> nodeNames = this.startServers(shardCount * replicationFactor);
- this.collectionToNodeNames.put(SOURCE_COLLECTION, nodeNames);
- this.createCollection(SOURCE_COLLECTION);
- this.waitForRecoveriesToFinish(SOURCE_COLLECTION, true);
- this.updateMappingsFromZk(SOURCE_COLLECTION);
- }
-
- /**
- * Clear the source collection. It will delete then create the collection through the collection API.
- * The collection will have a new fresh index, i.e., including a new update log.
- */
- protected void clearSourceCollection() throws Exception {
- this.deleteCollection(SOURCE_COLLECTION);
- this.waitForCollectionToDisappear(SOURCE_COLLECTION);
- this.createCollection(SOURCE_COLLECTION);
- this.waitForRecoveriesToFinish(SOURCE_COLLECTION, true);
- this.updateMappingsFromZk(SOURCE_COLLECTION);
- }
-
- /**
- * Starts the servers, saves and associates the node names to the target collection,
- * and finally creates the target collection.
- */
- private void createTargetCollection() throws Exception {
- List<String> nodeNames = this.startServers(shardCount * replicationFactor);
- this.collectionToNodeNames.put(TARGET_COLLECTION, nodeNames);
- this.createCollection(TARGET_COLLECTION);
- this.waitForRecoveriesToFinish(TARGET_COLLECTION, true);
- this.updateMappingsFromZk(TARGET_COLLECTION);
- }
-
- /**
- * Clear the source collection. It will delete then create the collection through the collection API.
- * The collection will have a new fresh index, i.e., including a new update log.
- */
- protected void clearTargetCollection() throws Exception {
- this.deleteCollection(TARGET_COLLECTION);
- this.waitForCollectionToDisappear(TARGET_COLLECTION);
- this.createCollection(TARGET_COLLECTION);
- this.waitForRecoveriesToFinish(TARGET_COLLECTION, true);
- this.updateMappingsFromZk(TARGET_COLLECTION);
- }
-
- /**
- * Create a new collection through the Collection API. It enforces the use of one max shard per node.
- * It will define the nodes to spread the new collection across by using the mapping {@link #collectionToNodeNames},
- * to ensure that a node will not host more than one core (which will create problem when trying to restart servers).
- */
- private void createCollection(String name) throws Exception {
- CloudSolrClient client = createCloudClient(null);
- try {
- // Create the target collection
- Map<String, List<Integer>> collectionInfos = new HashMap<>();
-
- StringBuilder sb = new StringBuilder();
- for (String nodeName : collectionToNodeNames.get(name)) {
- sb.append(nodeName);
- sb.append(',');
- }
- sb.deleteCharAt(sb.length() - 1);
-
- createCollection(collectionInfos, name, shardCount, replicationFactor, client, sb.toString());
- } finally {
- client.close();
- }
- }
-
- private CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
- String collectionName, int numShards, int replicationFactor,
- SolrClient client, String createNodeSetStr)
- throws SolrServerException, IOException {
- return createCollection(collectionInfos, collectionName,
- Utils.makeMap(
- NUM_SLICES, numShards,
- REPLICATION_FACTOR, replicationFactor,
- CREATE_NODE_SET, createNodeSetStr),
- client, "conf1");
- }
-
- private CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos, String collectionName,
- Map<String, Object> collectionProps, SolrClient client,
- String confSetName)
- throws SolrServerException, IOException {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set("action", CollectionParams.CollectionAction.CREATE.toString());
- for (Map.Entry<String, Object> entry : collectionProps.entrySet()) {
- if (entry.getValue() != null) params.set(entry.getKey(), String.valueOf(entry.getValue()));
- }
- Integer numShards = (Integer) collectionProps.get(OverseerCollectionMessageHandler.NUM_SLICES);
- if (numShards == null) {
- String shardNames = (String) collectionProps.get(OverseerCollectionMessageHandler.SHARDS_PROP);
- numShards = StrUtils.splitSmart(shardNames, ',').size();
- }
- Integer replicationFactor = (Integer) collectionProps.get(REPLICATION_FACTOR);
- if (replicationFactor == null) {
- replicationFactor = (Integer) OverseerCollectionMessageHandler.COLLECTION_PROPS_AND_DEFAULTS.get(REPLICATION_FACTOR);
- }
-
- if (confSetName != null) {
- params.set("collection.configName", confSetName);
- }
-
- List<Integer> list = new ArrayList<>();
- list.add(numShards);
- list.add(replicationFactor);
- if (collectionInfos != null) {
- collectionInfos.put(collectionName, list);
- }
- params.set("name", collectionName);
- @SuppressWarnings({"rawtypes"})
- SolrRequest request = new QueryRequest(params);
- request.setPath("/admin/collections");
-
- CollectionAdminResponse res = new CollectionAdminResponse();
- res.setResponse(client.request(request));
- return res;
- }
-
- /**
- * Delete a collection through the Collection API.
- */
- protected CollectionAdminResponse deleteCollection(String collectionName) throws Exception {
- SolrClient client = createCloudClient(null);
- CollectionAdminResponse res;
-
- try {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set("action", CollectionParams.CollectionAction.DELETE.toString());
- params.set("name", collectionName);
- QueryRequest request = new QueryRequest(params);
- request.setPath("/admin/collections");
-
- res = new CollectionAdminResponse();
- res.setResponse(client.request(request));
- } catch (Exception e) {
- log.warn("Error while deleting the collection {}", collectionName, e);
- return new CollectionAdminResponse();
- } finally {
- client.close();
- }
-
- return res;
- }
-
- private void waitForCollectionToDisappear(String collection) throws Exception {
- CloudSolrClient client = this.createCloudClient(null);
- try {
- client.connect();
- ZkStateReader zkStateReader = client.getZkStateReader();
- AbstractDistribZkTestBase.waitForCollectionToDisappear(collection, zkStateReader, true, 15);
- } finally {
- client.close();
- }
- }
-
- private void waitForRecoveriesToFinish(String collection, boolean verbose) throws Exception {
- CloudSolrClient client = this.createCloudClient(null);
- try {
- client.connect();
- ZkStateReader zkStateReader = client.getZkStateReader();
- super.waitForRecoveriesToFinish(collection, zkStateReader, verbose);
- } finally {
- client.close();
- }
- }
-
- /**
- * Asserts that the collection has the correct number of shards and replicas
- */
- protected void assertCollectionExpectations(String collectionName) throws Exception {
- CloudSolrClient client = this.createCloudClient(null);
- try {
- client.connect();
- ClusterState clusterState = client.getZkStateReader().getClusterState();
-
- assertTrue("Could not find new collection " + collectionName, clusterState.hasCollection(collectionName));
- Map<String, Slice> shards = clusterState.getCollection(collectionName).getSlicesMap();
- // did we find expectedSlices shards/shards?
- assertEquals("Found new collection " + collectionName + ", but mismatch on number of shards.", shardCount, shards.size());
- int totalShards = 0;
- for (String shardName : shards.keySet()) {
- totalShards += shards.get(shardName).getReplicas().size();
- }
- int expectedTotalShards = shardCount * replicationFactor;
- assertEquals("Found new collection " + collectionName + " with correct number of shards, but mismatch on number " +
- "of shards.", expectedTotalShards, totalShards);
- } finally {
- client.close();
- }
- }
-
- /**
- * Restart a server.
- */
- protected void restartServer(CloudJettyRunner server) throws Exception {
- // it seems we need to set the collection property to have the jetty properly restarted
- System.setProperty("collection", server.collection);
- JettySolrRunner jetty = server.jetty;
- jetty.stop();
- jetty.start();
- System.clearProperty("collection");
- waitForRecoveriesToFinish(server.collection, true);
- updateMappingsFromZk(server.collection); // must update the mapping as the core node name might have changed
- }
-
- /**
- * Restarts a list of servers.
- */
- protected void restartServers(List<CloudJettyRunner> servers) throws Exception {
- for (CloudJettyRunner server : servers) {
- this.restartServer(server);
- }
- }
-
- private List<JettySolrRunner> jettys = new ArrayList<>();
-
- /**
- * Creates and starts a given number of servers.
- */
- protected List<String> startServers(int nServer) throws Exception {
- String temporaryCollection = "tmp_collection";
-
- for (int i = 1; i <= nServer; i++) {
- // give everyone there own solrhome
- File jettyDir = createTempDir("jetty").toFile();
- jettyDir.mkdirs();
- setupJettySolrHome(jettyDir);
- JettySolrRunner jetty = createJetty(jettyDir, null, "shard" + i);
- jetty.start();
- jettys.add(jetty);
- }
-
- try (SolrClient client = createCloudClient(temporaryCollection)) {
- assertEquals(0, CollectionAdminRequest
- .createCollection(temporaryCollection, "conf1", shardCount, 1)
- .setCreateNodeSet("")
- .process(client).getStatus());
- for (int i = 0; i < jettys.size(); i++) {
- assertTrue(CollectionAdminRequest
- .addReplicaToShard(temporaryCollection, "shard"+((i % shardCount) + 1))
- .setNode(jettys.get(i).getNodeName())
- .process(client).isSuccess());
- }
- }
-
- ZkStateReader zkStateReader = jettys.get(0).getCoreContainer().getZkController().getZkStateReader();
-
- // now wait till we see the leader for each shard
- for (int i = 1; i <= shardCount; i++) {
- zkStateReader.getLeaderRetry(temporaryCollection, "shard" + i, 15000);
- }
-
- // store the node names
- List<String> nodeNames = new ArrayList<>();
- for (Slice shard : zkStateReader.getClusterState().getCollection(temporaryCollection).getSlices()) {
- for (Replica replica : shard.getReplicas()) {
- nodeNames.add(replica.getNodeName());
- }
- }
-
- this.waitForRecoveriesToFinish(temporaryCollection,zkStateReader, true);
- // delete the temporary collection - we will create our own collections later
- this.deleteCollection(temporaryCollection);
- this.waitForCollectionToDisappear(temporaryCollection);
- System.clearProperty("collection");
-
- return nodeNames;
- }
-
- @Override
- protected void destroyServers() throws Exception {
- for (JettySolrRunner runner : jettys) {
- try {
- runner.stop();
- } catch (Exception e) {
- log.error("", e);
- }
- }
-
- jettys.clear();
- }
-
- /**
- * Mapping from collection to jettys
- */
- protected Map<String, List<CloudJettyRunner>> cloudJettys = new HashMap<>();
-
- /**
- * Mapping from collection/shard to jettys
- */
- protected Map<String, Map<String, List<CloudJettyRunner>>> shardToJetty = new HashMap<>();
-
- /**
- * Mapping from collection/shard leader to jettys
- */
- protected Map<String, Map<String, CloudJettyRunner>> shardToLeaderJetty = new HashMap<>();
-
- /**
- * Updates the mappings between the jetty's instances and the zookeeper cluster state.
- */
- protected void updateMappingsFromZk(String collection) throws Exception {
- List<CloudJettyRunner> cloudJettys = new ArrayList<>();
- Map<String, List<CloudJettyRunner>> shardToJetty = new HashMap<>();
- Map<String, CloudJettyRunner> shardToLeaderJetty = new HashMap<>();
-
- CloudSolrClient cloudClient = this.createCloudClient(null);
- try {
- cloudClient.connect();
- ZkStateReader zkStateReader = cloudClient.getZkStateReader();
- ClusterState clusterState = zkStateReader.getClusterState();
- DocCollection coll = clusterState.getCollection(collection);
-
- for (JettySolrRunner jetty : jettys) {
- int port = jetty.getLocalPort();
- if (port == -1) {
- throw new RuntimeException("Cannot find the port for jetty");
- }
-
- nextJetty:
- for (Slice shard : coll.getSlices()) {
- Set<Map.Entry<String, Replica>> entries = shard.getReplicasMap().entrySet();
- for (Map.Entry<String, Replica> entry : entries) {
- Replica replica = entry.getValue();
- if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
- if (!shardToJetty.containsKey(shard.getName())) {
- shardToJetty.put(shard.getName(), new ArrayList<CloudJettyRunner>());
- }
- boolean isLeader = shard.getLeader() == replica;
- CloudJettyRunner cjr = new CloudJettyRunner(jetty, replica, collection, shard.getName(), entry.getKey());
- shardToJetty.get(shard.getName()).add(cjr);
- if (isLeader) {
- shardToLeaderJetty.put(shard.getName(), cjr);
- }
- cloudJettys.add(cjr);
- break nextJetty;
- }
- }
- }
- }
-
- List<CloudJettyRunner> oldRunners = this.cloudJettys.putIfAbsent(collection, cloudJettys);
- if (oldRunners != null) {
- // must close resources for the old entries
- for (CloudJettyRunner oldRunner : oldRunners) {
- IOUtils.closeQuietly(oldRunner.client);
- }
- }
-
- this.cloudJettys.put(collection, cloudJettys);
- this.shardToJetty.put(collection, shardToJetty);
- this.shardToLeaderJetty.put(collection, shardToLeaderJetty);
- } finally {
- cloudClient.close();
- }
- }
-
- /**
- * Wrapper around a {@link org.apache.solr.client.solrj.embedded.JettySolrRunner} to map the jetty
- * instance to various information of the cloud cluster, such as the collection and shard
- * that is served by the jetty instance, the node name, core node name, url, etc.
- */
- public static class CloudJettyRunner {
-
- public JettySolrRunner jetty;
- public String nodeName;
- public String coreNodeName;
- public String url;
- public SolrClient client;
- public Replica info;
- public String shard;
- public String collection;
-
- public CloudJettyRunner(JettySolrRunner jetty, Replica replica,
- String collection, String shard, String coreNodeName) {
- this.jetty = jetty;
- this.info = replica;
- this.collection = collection;
-
- Properties nodeProperties = jetty.getNodeProperties();
-
- // we need to update the jetty's shard so that it registers itself to the right shard when restarted
- this.shard = shard;
- nodeProperties.setProperty(CoreDescriptor.CORE_SHARD, this.shard);
-
- // we need to update the jetty's shard so that it registers itself under the right core name when restarted
- this.coreNodeName = coreNodeName;
- nodeProperties.setProperty(CoreDescriptor.CORE_NODE_NAME, this.coreNodeName);
-
- this.nodeName = replica.getNodeName();
-
- ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(info);
- this.url = coreNodeProps.getCoreUrl();
-
- // strip the trailing slash as this can cause issues when executing requests
- this.client = createNewSolrServer(this.url.substring(0, this.url.length() - 1));
- }
-
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + ((url == null) ? 0 : url.hashCode());
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj) return true;
- if (obj == null) return false;
- if (getClass() != obj.getClass()) return false;
- CloudJettyRunner other = (CloudJettyRunner) obj;
- if (url == null) {
- if (other.url != null) return false;
- } else if (!url.equals(other.url)) return false;
- return true;
- }
-
- @Override
- public String toString() {
- return "CloudJettyRunner [url=" + url + "]";
- }
-
- }
-
- protected static SolrClient createNewSolrServer(String baseUrl) {
- try {
- // setup the server...
- HttpSolrClient s = getHttpSolrClient(baseUrl, DEFAULT_CONNECTION_TIMEOUT);
- return s;
- } catch (Exception ex) {
- throw new RuntimeException(ex);
- }
- }
-
- protected void waitForBootstrapToComplete(String collectionName, String shardId) throws Exception {
- @SuppressWarnings({"rawtypes"})
- NamedList rsp;// we need to wait until bootstrap is complete otherwise the replicator thread will never start
- TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
- while (!timeOut.hasTimedOut()) {
- rsp = invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.BOOTSTRAP_STATUS);
- if (rsp.get(RESPONSE_STATUS).toString().equals(COMPLETED)) {
- break;
- }
- Thread.sleep(1000);
- }
- }
-
- protected void waitForReplicationToComplete(String collectionName, String shardId) throws Exception {
- int cnt = 15;
- while (cnt > 0) {
- log.info("Checking queue size @ {}:{}", collectionName, shardId);
- long size = this.getQueueSize(collectionName, shardId);
- if (size == 0) { // if we received -1, it means that the log reader is not yet initialised, we should wait
- return;
- }
- log.info("Waiting for replication to complete. Queue size: {} @ {}:{}", size, collectionName, shardId);
- cnt--;
- Thread.sleep(1000); // wait a bit for the replication to complete
- }
- throw new RuntimeException("Timeout waiting for CDCR replication to complete @" + collectionName + ":" + shardId);
- }
-
- protected long getQueueSize(String collectionName, String shardId) throws Exception {
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = this.invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.QUEUES);
- @SuppressWarnings({"rawtypes"})
- NamedList host = (NamedList) ((NamedList) rsp.get(CdcrParams.QUEUES)).getVal(0);
- @SuppressWarnings({"rawtypes"})
- NamedList status = (NamedList) host.get(TARGET_COLLECTION);
- return (Long) status.get(CdcrParams.QUEUE_SIZE);
- }
-
- protected CollectionInfo collectInfo(String collection) throws Exception {
- CollectionInfo info = new CollectionInfo(collection);
- for (String shard : shardToJetty.get(collection).keySet()) {
- List<CloudJettyRunner> jettyRunners = shardToJetty.get(collection).get(shard);
- for (CloudJettyRunner jettyRunner : jettyRunners) {
- for (SolrCore core : jettyRunner.jetty.getCoreContainer().getCores()) {
- info.addCore(core, shard, shardToLeaderJetty.get(collection).containsValue(jettyRunner));
- }
- }
- }
-
- return info;
- }
-
- protected static class CollectionInfo {
-
- List<CoreInfo> coreInfos = new ArrayList<>();
-
- String collection;
-
- CollectionInfo(String collection) {
- this.collection = collection;
- }
-
- /**
- * @return Returns a map shard -> list of cores
- */
- Map<String, List<CoreInfo>> getShardToCoresMap() {
- Map<String, List<CoreInfo>> map = new HashMap<>();
- for (CoreInfo info : coreInfos) {
- List<CoreInfo> list = map.get(info.shard);
- if (list == null) {
- list = new ArrayList<>();
- map.put(info.shard, list);
- }
- list.add(info);
- }
- return map;
- }
-
- CoreInfo getLeader(String shard) {
- List<CoreInfo> coreInfos = getShardToCoresMap().get(shard);
- for (CoreInfo info : coreInfos) {
- if (info.isLeader) {
- return info;
- }
- }
- assertTrue(String.format(Locale.ENGLISH, "There is no leader for collection %s shard %s", collection, shard), false);
- return null;
- }
-
- List<CoreInfo> getReplicas(String shard) {
- List<CoreInfo> coreInfos = getShardToCoresMap().get(shard);
- coreInfos.remove(getLeader(shard));
- return coreInfos;
- }
-
- void addCore(SolrCore core, String shard, boolean isLeader) throws Exception {
- CoreInfo info = new CoreInfo();
- info.collectionName = core.getName();
- info.shard = shard;
- info.isLeader = isLeader;
- info.ulogDir = core.getUpdateHandler().getUpdateLog().getLogDir();
-
- this.coreInfos.add(info);
- }
-
- public static class CoreInfo {
- String collectionName;
- String shard;
- boolean isLeader;
- String ulogDir;
- }
-
- }
-
-}
-
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBidirectionalTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBidirectionalTest.java
deleted file mode 100644
index 7f1db84..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBidirectionalTest.java
+++ /dev/null
@@ -1,244 +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.cdcr;
-
-import java.lang.invoke.MethodHandles;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.MiniSolrCloudCluster;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.handler.CdcrParams;
-import org.apache.solr.util.TimeOut;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class CdcrBidirectionalTest extends SolrTestCaseJ4 {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- @Test
- @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-12524")
- public void testBiDir() throws Exception {
- MiniSolrCloudCluster cluster2 = new MiniSolrCloudCluster(1, createTempDir("cdcr-cluster2"), buildJettyConfig("/solr"));
- MiniSolrCloudCluster cluster1 = new MiniSolrCloudCluster(1, createTempDir("cdcr-cluster1"), buildJettyConfig("/solr"));
- try {
- if (log.isInfoEnabled()) {
- log.info("cluster2 zkHost = {}", cluster2.getZkServer().getZkAddress());
- }
- System.setProperty("cdcr.cluster2.zkHost", cluster2.getZkServer().getZkAddress());
-
- if (log.isInfoEnabled()) {
- log.info("cluster1 zkHost = {}", cluster1.getZkServer().getZkAddress());
- }
- System.setProperty("cdcr.cluster1.zkHost", cluster1.getZkServer().getZkAddress());
-
-
- cluster1.uploadConfigSet(configset("cdcr-cluster1"), "cdcr-cluster1");
- CollectionAdminRequest.createCollection("cdcr-cluster1", "cdcr-cluster1", 2, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(cluster1.getSolrClient());
- CloudSolrClient cluster1SolrClient = cluster1.getSolrClient();
- cluster1SolrClient.setDefaultCollection("cdcr-cluster1");
-
- cluster2.uploadConfigSet(configset("cdcr-cluster2"), "cdcr-cluster2");
- CollectionAdminRequest.createCollection("cdcr-cluster2", "cdcr-cluster2", 2, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(cluster2.getSolrClient());
- CloudSolrClient cluster2SolrClient = cluster2.getSolrClient();
- cluster2SolrClient.setDefaultCollection("cdcr-cluster2");
-
- UpdateRequest req = null;
-
- CdcrTestsUtil.cdcrStart(cluster1SolrClient);
- Thread.sleep(2000);
-
- // ADD operation on cluster 1
- int docs = (TEST_NIGHTLY ? 100 : 10);
- int numDocs_c1 = 0;
- for (int k = 0; k < docs; k++) {
- req = new UpdateRequest();
- for (; numDocs_c1 < (k + 1) * 100; numDocs_c1++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "cluster1_" + numDocs_c1);
- doc.addField("xyz", numDocs_c1);
- req.add(doc);
- }
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- log.info("Adding {} docs with commit=true, numDocs={}", docs, numDocs_c1);
- req.process(cluster1SolrClient);
- }
-
- QueryResponse response = cluster1SolrClient.query(new SolrQuery("*:*"));
- assertEquals("cluster 1 docs mismatch", numDocs_c1, response.getResults().getNumFound());
-
- assertEquals("cluster 2 docs mismatch", numDocs_c1, CdcrTestsUtil.waitForClusterToSync(numDocs_c1, cluster2SolrClient));
-
- CdcrTestsUtil.cdcrStart(cluster2SolrClient); // FULL BI-DIRECTIONAL CDCR FORWARDING ON
- Thread.sleep(2000);
-
- // ADD operation on cluster 2
- int numDocs_c2 = 0;
- for (int k = 0; k < docs; k++) {
- req = new UpdateRequest();
- for (; numDocs_c2 < (k + 1) * 100; numDocs_c2++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "cluster2_" + numDocs_c2);
- doc.addField("xyz", numDocs_c2);
- req.add(doc);
- }
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- log.info("Adding {} docs with commit=true, numDocs= {}", docs, numDocs_c2);
- req.process(cluster2SolrClient);
- }
-
- int numDocs = numDocs_c1 + numDocs_c2;
-
- response = cluster2SolrClient.query(new SolrQuery("*:*"));
- assertEquals("cluster 2 docs mismatch", numDocs, response.getResults().getNumFound());
-
- assertEquals("cluster 1 docs mismatch", numDocs, CdcrTestsUtil.waitForClusterToSync(numDocs, cluster1SolrClient));
-
- // logging cdcr clusters queue response
- response = CdcrTestsUtil.getCdcrQueue(cluster1SolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr cluster1 queue response: {}", response.getResponse());
- }
- response = CdcrTestsUtil.getCdcrQueue(cluster2SolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr cluster2 queue response: {}", response.getResponse());
- }
-
- // lets find and keep the maximum version assigned by cluster1 & cluster2 across all our updates
-
- long maxVersion_c1 = Math.min((long)CdcrTestsUtil.getFingerPrintMaxVersion(cluster1SolrClient, "shard1", numDocs),
- (long)CdcrTestsUtil.getFingerPrintMaxVersion(cluster1SolrClient, "shard2", numDocs));
- long maxVersion_c2 = Math.min((long)CdcrTestsUtil.getFingerPrintMaxVersion(cluster2SolrClient, "shard1", numDocs),
- (long)CdcrTestsUtil.getFingerPrintMaxVersion(cluster2SolrClient, "shard2", numDocs));
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.COLLECTIONCHECKPOINT.toString());
- params.set(CommonParams.QT, "/cdcr");
- response = cluster2SolrClient.query(params);
- Long checkpoint_2 = (Long) response.getResponse().get(CdcrParams.CHECKPOINT);
- assertNotNull(checkpoint_2);
-
- params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.COLLECTIONCHECKPOINT.toString());
- params.set(CommonParams.QT, "/cdcr");
- response = cluster1SolrClient.query(params);
- Long checkpoint_1 = (Long) response.getResponse().get(CdcrParams.CHECKPOINT);
- assertNotNull(checkpoint_1);
-
- log.info("v1: {}\tv2: {}\tcheckpoint1: {}\tcheckpoint2: {}"
- , maxVersion_c1, maxVersion_c2, checkpoint_1, checkpoint_2);
-
- assertEquals("COLLECTIONCHECKPOINT from cluster2 should have returned the maximum " +
- "version across all updates made to cluster1", maxVersion_c1, checkpoint_2.longValue());
- assertEquals("COLLECTIONCHECKPOINT from cluster1 should have returned the maximum " +
- "version across all updates made to cluster2", maxVersion_c2, checkpoint_1.longValue());
- assertEquals("max versions of updates in both clusters should be same", maxVersion_c1, maxVersion_c2);
-
- // DELETE BY QUERY
- String deleteByQuery = "id:cluster1_" +String.valueOf(random().nextInt(numDocs_c1));
- response = cluster1SolrClient.query(new SolrQuery(deleteByQuery));
- assertEquals("should match exactly one doc", 1, response.getResults().getNumFound());
- cluster1SolrClient.deleteByQuery(deleteByQuery);
- cluster1SolrClient.commit();
- numDocs--;
- numDocs_c1--;
-
- response = cluster1SolrClient.query(new SolrQuery("*:*"));
- assertEquals("cluster 1 docs mismatch", numDocs, response.getResults().getNumFound());
- assertEquals("cluster 2 docs mismatch", numDocs, CdcrTestsUtil.waitForClusterToSync(numDocs, cluster2SolrClient));
-
- // DELETE BY ID
- SolrInputDocument doc;
- String delete_id_query = "cluster2_" + random().nextInt(numDocs_c2);
- cluster2SolrClient.deleteById(delete_id_query);
- cluster2SolrClient.commit();
- numDocs--;
- numDocs_c2--;
- response = cluster2SolrClient.query(new SolrQuery("*:*"));
- assertEquals("cluster 2 docs mismatch", numDocs, response.getResults().getNumFound());
- assertEquals("cluster 1 docs mismatch", numDocs, CdcrTestsUtil.waitForClusterToSync(numDocs, cluster1SolrClient));
-
- // ATOMIC UPDATES
- req = new UpdateRequest();
- doc = new SolrInputDocument();
- String atomicFieldName = "abc";
- String atomicUpdateId = "cluster2_" + random().nextInt(numDocs_c2);
- doc.addField("id", atomicUpdateId);
- doc.addField("xyz", ImmutableMap.of("delete", ""));
- doc.addField(atomicFieldName, ImmutableMap.of("set", "ABC"));
- req.add(doc);
- req.process(cluster2SolrClient);
- cluster2SolrClient.commit();
-
- String atomicQuery = "id:" + atomicUpdateId;
- response = cluster2SolrClient.query(new SolrQuery(atomicQuery));
- assertEquals("cluster 2 wrong doc", "ABC", response.getResults().get(0).get(atomicFieldName));
- assertEquals("cluster 1 wrong doc", "ABC", getDocFieldValue(cluster1SolrClient, atomicQuery, "ABC", atomicFieldName ));
-
-
- // logging cdcr clusters queue response
- response = CdcrTestsUtil.getCdcrQueue(cluster1SolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr cluster1 queue response at end of testcase: {}", response.getResponse());
- }
- response = CdcrTestsUtil.getCdcrQueue(cluster2SolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr cluster2 queue response at end of testcase: {}", response.getResponse());
- }
-
- CdcrTestsUtil.cdcrStop(cluster1SolrClient);
- CdcrTestsUtil.cdcrStop(cluster2SolrClient);
- } finally {
- if (cluster1 != null) {
- cluster1.shutdown();
- }
- if (cluster2 != null) {
- cluster2.shutdown();
- }
- }
- }
-
- private String getDocFieldValue(CloudSolrClient clusterSolrClient, String query, String match, String field) throws Exception {
- TimeOut waitTimeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
- while (!waitTimeOut.hasTimedOut()) {
- clusterSolrClient.commit();
- QueryResponse response = clusterSolrClient.query(new SolrQuery(query));
- if (response.getResults().size() > 0 && match.equals(response.getResults().get(0).get(field))) {
- return (String) response.getResults().get(0).get(field);
- }
- Thread.sleep(1000);
- }
- return null;
- }
-}
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBootstrapTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBootstrapTest.java
deleted file mode 100644
index 34d8287..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrBootstrapTest.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.cloud.cdcr;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.LinkedHashMap;
-
-import org.apache.lucene.store.FSDirectory;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.AbstractDistribZkTestBase;
-import org.apache.solr.cloud.MiniSolrCloudCluster;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.handler.CdcrParams;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class CdcrBootstrapTest extends SolrTestCaseJ4 {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- /**
- * Starts a source cluster with no CDCR configuration, indexes enough documents such that
- * the at least one old tlog is closed and thrown away so that the source cluster does not have
- * all updates available in tlogs only.
- * <p>
- * Then we start a target cluster with CDCR configuration and we change the source cluster configuration
- * to use CDCR (i.e. CdcrUpdateLog, CdcrRequestHandler and CdcrUpdateProcessor) and restart it.
- * <p>
- * We test that all updates eventually make it to the target cluster and that the collectioncheckpoint
- * call returns the same version as the last update indexed on the source.
- */
- @Test
- // commented 4-Sep-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2-Aug-2018
- // commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
- public void testConvertClusterToCdcrAndBootstrap() throws Exception {
- // start the target first so that we know its zkhost
- MiniSolrCloudCluster target = new MiniSolrCloudCluster(1, createTempDir("cdcr-target"), buildJettyConfig("/solr"));
- try {
- if (log.isInfoEnabled()) {
- log.info("Target zkHost = {}", target.getZkServer().getZkAddress());
- }
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
-
- // start a cluster with no cdcr
- MiniSolrCloudCluster source = new MiniSolrCloudCluster(1, createTempDir("cdcr-source"), buildJettyConfig("/solr"));
- try {
- source.uploadConfigSet(configset("cdcr-source-disabled"), "cdcr-source");
-
- // create a collection with the cdcr-source-disabled configset
- CollectionAdminRequest.createCollection("cdcr-source", "cdcr-source", 1, 1)
- // todo investigate why this is necessary??? because by default it selects a ram directory which deletes the tlogs on reloads?
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- source.waitForActiveCollection("cdcr-source", 1, 1);
- CloudSolrClient sourceSolrClient = source.getSolrClient();
- int docs = (TEST_NIGHTLY ? 100 : 10);
- int numDocs = indexDocs(sourceSolrClient, "cdcr-source", docs);
-
- QueryResponse response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- // lets find and keep the maximum version assigned by source cluster across all our updates
- long maxVersion = Long.MIN_VALUE;
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.QT, "/get");
- params.set("getVersions", numDocs);
- params.set("fingerprint", true);
- response = sourceSolrClient.query(params);
- maxVersion = (long)(((LinkedHashMap)response.getResponse().get("fingerprint")).get("maxVersionEncountered"));
-
-// upload the cdcr-enabled config and restart source cluster
- source.uploadConfigSet(configset("cdcr-source"), "cdcr-source");
- JettySolrRunner runner = source.stopJettySolrRunner(0);
- source.waitForJettyToStop(runner);
-
- source.startJettySolrRunner(runner);
- source.waitForAllNodes(30);
- assertTrue(runner.isRunning());
- AbstractDistribZkTestBase.waitForRecoveriesToFinish("cdcr-source", source.getSolrClient().getZkStateReader(), true, true, 330);
-
- response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("Document mismatch on source after restart", numDocs, response.getResults().getNumFound());
-
- // setup the target cluster
- target.uploadConfigSet(configset("cdcr-target"), "cdcr-target");
- CollectionAdminRequest.createCollection("cdcr-target", "cdcr-target", 1, 2)
- .process(target.getSolrClient());
- target.waitForActiveCollection("cdcr-target", 1, 2);
- CloudSolrClient targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
- Thread.sleep(6000);
-
- CdcrTestsUtil.cdcrStart(targetSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
-
- response = CdcrTestsUtil.getCdcrQueue(sourceSolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr queue response: {}", response.getResponse());
- }
- long foundDocs = CdcrTestsUtil.waitForClusterToSync(numDocs, targetSolrClient);
- assertEquals("Document mismatch on target after sync", numDocs, foundDocs);
- assertTrue(CdcrTestsUtil.assertShardInSync("cdcr-target", "shard1", targetSolrClient)); // with more than 1 replica
-
- params = new ModifiableSolrParams();
- params.set(CommonParams.ACTION, CdcrParams.CdcrAction.COLLECTIONCHECKPOINT.toString());
- params.set(CommonParams.QT, "/cdcr");
- response = targetSolrClient.query(params);
- Long checkpoint = (Long) response.getResponse().get(CdcrParams.CHECKPOINT);
- assertNotNull(checkpoint);
- assertEquals("COLLECTIONCHECKPOINT from target cluster should have returned the maximum " +
- "version across all updates made to source", maxVersion, checkpoint.longValue());
- } finally {
- source.shutdown();
- }
- } finally {
- target.shutdown();
- }
- }
-
- private int indexDocs(CloudSolrClient sourceSolrClient, String collection, int batches) throws IOException, SolrServerException {
- sourceSolrClient.setDefaultCollection(collection);
- int numDocs = 0;
- for (int k = 0; k < batches; k++) {
- UpdateRequest req = new UpdateRequest();
- for (; numDocs < (k + 1) * 100; numDocs++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "source_" + numDocs);
- doc.addField("xyz", numDocs);
- req.add(doc);
- }
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- req.process(sourceSolrClient);
- }
- log.info("Adding numDocs={}", numDocs);
- return numDocs;
- }
- /**
- * This test start cdcr source, adds data,starts target cluster, verifies replication,
- * stops cdcr replication and buffering, adds more data, re-enables cdcr and verify replication
- */
- public void testBootstrapWithSourceCluster() throws Exception {
- // start the target first so that we know its zkhost
- MiniSolrCloudCluster target = new MiniSolrCloudCluster(1, createTempDir("cdcr-target"), buildJettyConfig("/solr"));
- try {
- System.out.println("Target zkHost = " + target.getZkServer().getZkAddress());
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
-
- MiniSolrCloudCluster source = new MiniSolrCloudCluster(1, createTempDir("cdcr-source"), buildJettyConfig("/solr"));
- try {
- source.uploadConfigSet(configset("cdcr-source"), "cdcr-source");
-
- CollectionAdminRequest.createCollection("cdcr-source", "cdcr-source", 1, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- source.waitForActiveCollection("cdcr-source", 1, 1);
-
- CloudSolrClient sourceSolrClient = source.getSolrClient();
- int docs = (TEST_NIGHTLY ? 100 : 10);
- int numDocs = indexDocs(sourceSolrClient, "cdcr-source", docs);
-
- QueryResponse response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- // setup the target cluster
- target.uploadConfigSet(configset("cdcr-target"), "cdcr-target");
- CollectionAdminRequest.createCollection("cdcr-target", "cdcr-target", 1, 1)
- .process(target.getSolrClient());
- target.waitForActiveCollection("cdcr-target", 1, 1);
- CloudSolrClient targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
-
- CdcrTestsUtil.cdcrStart(targetSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
-
- response = CdcrTestsUtil.getCdcrQueue(sourceSolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr queue response: {}", response.getResponse());
- }
- long foundDocs = CdcrTestsUtil.waitForClusterToSync(numDocs, targetSolrClient);
- assertEquals("Document mismatch on target after sync", numDocs, foundDocs);
-
- int total_tlogs_in_index = FSDirectory.open(target.getBaseDir().resolve("node1").
- resolve("cdcr-target_shard1_replica_n1").resolve("data").
- resolve("tlog")).listAll().length;
-
- assertEquals("tlogs count should be ZERO",0, total_tlogs_in_index);
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
- CdcrTestsUtil.cdcrDisableBuffer(sourceSolrClient);
-
- int c = 0;
- for (int k = 0; k < 10; k++) {
- UpdateRequest req = new UpdateRequest();
- for (; c < (k + 1) * 100; c++, numDocs++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "source_" + numDocs);
- doc.addField("xyz", numDocs);
- req.add(doc);
- }
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- log.info("Adding 100 docs with commit=true, numDocs={}", numDocs);
- req.process(sourceSolrClient);
- }
-
- response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
- CdcrTestsUtil.cdcrEnableBuffer(sourceSolrClient);
-
- foundDocs = CdcrTestsUtil.waitForClusterToSync(numDocs, targetSolrClient);
- assertEquals("Document mismatch on target after sync", numDocs, foundDocs);
-
- } finally {
- source.shutdown();
- }
- } finally {
- target.shutdown();
- }
- }
-
- /**
- * This test successfully validates the follower nodes at target copies content
- * from their respective leaders
- */
- public void testBootstrapWithMultipleReplicas() throws Exception {
- // start the target first so that we know its zkhost
- MiniSolrCloudCluster target = new MiniSolrCloudCluster(3, createTempDir("cdcr-target"), buildJettyConfig("/solr"));
- try {
- System.out.println("Target zkHost = " + target.getZkServer().getZkAddress());
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
-
- MiniSolrCloudCluster source = new MiniSolrCloudCluster(3, createTempDir("cdcr-source"), buildJettyConfig("/solr"));
- try {
- source.uploadConfigSet(configset("cdcr-source"), "cdcr-source");
-
- CollectionAdminRequest.createCollection("cdcr-source", "cdcr-source", 1, 3)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- source.waitForActiveCollection("cdcr-source", 1, 3);
-
- CloudSolrClient sourceSolrClient = source.getSolrClient();
- int docs = (TEST_NIGHTLY ? 100 : 10);
- int numDocs = indexDocs(sourceSolrClient, "cdcr-source", docs);
-
- QueryResponse response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- // setup the target cluster
- target.uploadConfigSet(configset("cdcr-target"), "cdcr-target");
- CollectionAdminRequest.createCollection("cdcr-target", "cdcr-target", 1, 3)
- .process(target.getSolrClient());
- target.waitForActiveCollection("cdcr-target", 1, 3);
- CloudSolrClient targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
-
- CdcrTestsUtil.cdcrStart(targetSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
-
- response = CdcrTestsUtil.getCdcrQueue(sourceSolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr queue response: {}", response.getResponse());
- }
- long foundDocs = CdcrTestsUtil.waitForClusterToSync(numDocs, targetSolrClient);
- assertEquals("Document mismatch on target after sync", numDocs, foundDocs);
- assertTrue("leader followers didnt' match", CdcrTestsUtil.assertShardInSync("cdcr-target", "shard1", targetSolrClient)); // with more than 1 replica
-
- } finally {
- source.shutdown();
- }
- } finally {
- target.shutdown();
- }
- }
-
- // 29-June-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
- @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 6-Sep-2018
- @Test
- @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-12028")
- public void testBootstrapWithContinousIndexingOnSourceCluster() throws Exception {
- // start the target first so that we know its zkhost
- MiniSolrCloudCluster target = new MiniSolrCloudCluster(1, createTempDir("cdcr-target"), buildJettyConfig("/solr"));
- try {
- if (log.isInfoEnabled()) {
- log.info("Target zkHost = {}", target.getZkServer().getZkAddress());
- }
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
-
- MiniSolrCloudCluster source = new MiniSolrCloudCluster(1, createTempDir("cdcr-source"), buildJettyConfig("/solr"));
- try {
- source.uploadConfigSet(configset("cdcr-source"), "cdcr-source");
-
- CollectionAdminRequest.createCollection("cdcr-source", "cdcr-source", 1, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- source.waitForActiveCollection("cdcr-source", 1, 1);
- CloudSolrClient sourceSolrClient = source.getSolrClient();
- int docs = (TEST_NIGHTLY ? 100 : 10);
- int numDocs = indexDocs(sourceSolrClient, "cdcr-source", docs);
-
- QueryResponse response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- // setup the target cluster
- target.uploadConfigSet(configset("cdcr-target"), "cdcr-target");
- CollectionAdminRequest.createCollection("cdcr-target", "cdcr-target", 1, 1)
- .process(target.getSolrClient());
- target.waitForActiveCollection("cdcr-target", 1, 1);
- CloudSolrClient targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
- Thread.sleep(1000);
-
- CdcrTestsUtil.cdcrStart(targetSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
- int c = 0;
- for (int k = 0; k < docs; k++) {
- UpdateRequest req = new UpdateRequest();
- for (; c < (k + 1) * 100; c++, numDocs++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "source_" + numDocs);
- doc.addField("xyz", numDocs);
- req.add(doc);
- }
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- log.info("Adding {} docs with commit=true, numDocs={}", docs, numDocs);
- req.process(sourceSolrClient);
- }
-
- response = sourceSolrClient.query(new SolrQuery("*:*"));
- assertEquals("", numDocs, response.getResults().getNumFound());
-
- response = CdcrTestsUtil.getCdcrQueue(sourceSolrClient);
- if (log.isInfoEnabled()) {
- log.info("Cdcr queue response: {}", response.getResponse());
- }
- long foundDocs = CdcrTestsUtil.waitForClusterToSync(numDocs, targetSolrClient);
- assertEquals("Document mismatch on target after sync", numDocs, foundDocs);
-
- } finally {
- source.shutdown();
- }
- } finally {
- target.shutdown();
- }
- }
-}
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrOpsAndBoundariesTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrOpsAndBoundariesTest.java
deleted file mode 100644
index 2eb8d9f..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrOpsAndBoundariesTest.java
+++ /dev/null
@@ -1,332 +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.cdcr;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.MiniSolrCloudCluster;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.handler.CdcrParams;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.Nightly;
-
-@Nightly // test is too long for non nightly
-public class CdcrOpsAndBoundariesTest extends SolrTestCaseJ4 {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- MiniSolrCloudCluster target, source;
- CloudSolrClient sourceSolrClient, targetSolrClient;
- private static String SOURCE_COLLECTION = "cdcr-source";
- private static String TARGET_COLLECTION = "cdcr-target";
- private static String ALL_Q = "*:*";
-
- @Before
- public void before() throws Exception {
- target = new MiniSolrCloudCluster(1, createTempDir(TARGET_COLLECTION), buildJettyConfig("/solr"));
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
- source = new MiniSolrCloudCluster(1, createTempDir(SOURCE_COLLECTION), buildJettyConfig("/solr"));
- }
-
- @After
- public void after() throws Exception {
- if (null != target) {
- target.shutdown();
- target = null;
- }
- if (null != source) {
- source.shutdown();
- source = null;
- }
- }
-
- /**
- * Check the ops statistics.
- */
- @Test
- @SuppressWarnings({"rawtypes"})
- public void testOps() throws Exception {
- createCollections();
-
- // Start CDCR
- CdcrTestsUtil.cdcrRestart(sourceSolrClient);
-
- // Index documents
- CdcrTestsUtil.indexRandomDocs(100, sourceSolrClient);
- double opsAll = 0.0;
- NamedList ops = null;
-
- // calculate ops
- int itr = 10;
- while (itr-- > 0 && opsAll == 0.0) {
- NamedList rsp = CdcrTestsUtil.invokeCdcrAction(sourceSolrClient, CdcrParams.CdcrAction.OPS).getResponse();
- NamedList collections = (NamedList) ((NamedList) rsp.get(CdcrParams.OPERATIONS_PER_SECOND)).getVal(0);
- ops = (NamedList) collections.get(TARGET_COLLECTION);
- opsAll = (Double) ops.get(CdcrParams.COUNTER_ALL);
- Thread.sleep(250); // wait for cdcr to complete and check
- }
- // asserts ops values
- double opsAdds = (Double) ops.get(CdcrParams.COUNTER_ADDS);
- assertTrue(opsAll > 0);
- assertTrue(opsAdds > 0);
- double opsDeletes = (Double) ops.get(CdcrParams.COUNTER_DELETES);
- assertEquals(0, opsDeletes, 0);
-
- // Delete 10 documents: 10-19
- List<String> ids;
- for (int id = 0; id < 50; id++) {
- ids = new ArrayList<>();
- ids.add(Integer.toString(id));
- sourceSolrClient.deleteById(ids, 1);
- int dbq_id = 50 + id;
- sourceSolrClient.deleteByQuery("id:" + dbq_id, 1);
- }
-
- itr = 10;
- while (itr-- > 0) {
- NamedList rsp = CdcrTestsUtil.invokeCdcrAction(sourceSolrClient, CdcrParams.CdcrAction.OPS).getResponse();
- NamedList collections = (NamedList) ((NamedList) rsp.get(CdcrParams.OPERATIONS_PER_SECOND)).getVal(0);
- ops = (NamedList) collections.get(TARGET_COLLECTION);
- opsAll = (Double) ops.get(CdcrParams.COUNTER_ALL);
- Thread.sleep(250); // wait for cdcr to complete and check
- }
- // asserts ops values
- opsAdds = (Double) ops.get(CdcrParams.COUNTER_ADDS);
- opsDeletes = (Double) ops.get(CdcrParams.COUNTER_DELETES);
- assertTrue(opsAll > 0);
- assertTrue(opsAdds > 0);
- assertTrue(opsDeletes > 0);
-
- deleteCollections();
- }
-
- @Test
- public void testTargetCollectionNotAvailable() throws Exception {
- createCollections();
-
- // send start action to first shard
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
-
- assertNotSame(null, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- // sleep for a bit to ensure that replicator threads are started
- Thread.sleep(3000);
-
- target.deleteAllCollections();
-
- CdcrTestsUtil.indexRandomDocs(6, sourceSolrClient);
- assertEquals(6L, sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound());
-
- // we need to wait until the replicator thread is triggered
- int cnt = 15; // timeout after 15 seconds
- AssertionError lastAssertionError = null;
- while (cnt > 0) {
- try {
- QueryResponse rsp = CdcrTestsUtil.invokeCdcrAction(sourceSolrClient, CdcrParams.CdcrAction.ERRORS);
- @SuppressWarnings({"rawtypes"})
- NamedList collections = (NamedList) ((NamedList) rsp.getResponse().get(CdcrParams.ERRORS)).getVal(0);
- @SuppressWarnings({"rawtypes"})
- NamedList errors = (NamedList) collections.get(TARGET_COLLECTION);
- assertTrue(0 < (Long) errors.get(CdcrParams.CONSECUTIVE_ERRORS));
- @SuppressWarnings({"rawtypes"})
- NamedList lastErrors = (NamedList) errors.get(CdcrParams.LAST);
- assertNotNull(lastErrors);
- assertTrue(0 < lastErrors.size());
- deleteCollections();
- return;
- } catch (AssertionError e) {
- lastAssertionError = e;
- cnt--;
- Thread.sleep(1000);
- }
- }
-
- deleteCollections();
- throw new AssertionError("Timeout while trying to assert replication errors", lastAssertionError);
- }
-
- @Test
- public void testReplicationStartStop() throws Exception {
- createCollections();
-
- CdcrTestsUtil.indexRandomDocs(10, sourceSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
-
- assertEquals(10, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
-
- CdcrTestsUtil.indexRandomDocs(110, sourceSolrClient);
-
- // Start again CDCR, the source cluster should reinitialise its log readers
- // with the latest checkpoints
-
- CdcrTestsUtil.cdcrRestart(sourceSolrClient);
-
- assertEquals(110, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- deleteCollections();
- }
-
- /**
- * Check that batch updates with deletes
- */
- @Test
- public void testBatchAddsWithDelete() throws Exception {
- createCollections();
-
- // Start CDCR
- CdcrTestsUtil.cdcrRestart(sourceSolrClient);
- // Index 50 documents
- CdcrTestsUtil.indexRandomDocs(50, sourceSolrClient);
-
- // Delete 10 documents: 10-19
- List<String> ids = new ArrayList<>();
- for (int id = 10; id < 20; id++) {
- ids.add(Integer.toString(id));
- }
- sourceSolrClient.deleteById(ids, 10);
-
- CdcrTestsUtil.indexRandomDocs(50, 60, sourceSolrClient);
-
- // Delete 1 document: 50
- ids = new ArrayList<>();
- ids.add(Integer.toString(50));
- sourceSolrClient.deleteById(ids, 10);
-
- CdcrTestsUtil.indexRandomDocs(60, 70, sourceSolrClient);
-
- assertEquals(59, CdcrTestsUtil.waitForClusterToSync(59, sourceSolrClient));
- assertEquals(59, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- deleteCollections();
- }
-
- /**
- * Checks that batches are correctly constructed when batch boundaries are reached.
- */
- @Test
- public void testBatchBoundaries() throws Exception {
- createCollections();
-
- // Start CDCR
- CdcrTestsUtil.cdcrRestart(sourceSolrClient);
-
- log.info("Indexing documents");
-
- CdcrTestsUtil.indexRandomDocs(1000, sourceSolrClient);
-
- assertEquals(1000, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- deleteCollections();
- }
-
- /**
- * Check resilience of replication with delete by query executed on targets
- */
- @Test
- public void testResilienceWithDeleteByQueryOnTarget() throws Exception {
- createCollections();
-
- // Start CDCR
- CdcrTestsUtil.cdcrRestart(sourceSolrClient);
-
- CdcrTestsUtil.indexRandomDocs(50, sourceSolrClient);
-
- assertEquals(50, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- sourceSolrClient.deleteByQuery(ALL_Q, 1);
-
- assertEquals(0, CdcrTestsUtil.waitForClusterToSync(0, sourceSolrClient));
- assertEquals(0, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- CdcrTestsUtil.indexRandomDocs(51, 101, sourceSolrClient);
-
- assertEquals(50, CdcrTestsUtil.waitForClusterToSync
- (sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound(), targetSolrClient));
-
- targetSolrClient.deleteByQuery(ALL_Q, 1);
-
- assertEquals(50, sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound());
- assertEquals(0, CdcrTestsUtil.waitForClusterToSync(0, targetSolrClient));
-
- CdcrTestsUtil.indexRandomDocs(102, 152, sourceSolrClient);
-
- assertEquals(100, sourceSolrClient.query(new SolrQuery(ALL_Q)).getResults().getNumFound());
- assertEquals(50, CdcrTestsUtil.waitForClusterToSync(50, targetSolrClient));
-
- deleteCollections();
- }
-
- private void createSourceCollection() throws Exception {
- source.uploadConfigSet(configset(SOURCE_COLLECTION), SOURCE_COLLECTION);
- CollectionAdminRequest.createCollection(SOURCE_COLLECTION, SOURCE_COLLECTION, 1, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- Thread.sleep(1000);
- sourceSolrClient = source.getSolrClient();
- sourceSolrClient.setDefaultCollection(SOURCE_COLLECTION);
- }
-
- private void createTargetCollection() throws Exception {
- target.uploadConfigSet(configset(TARGET_COLLECTION), TARGET_COLLECTION);
- CollectionAdminRequest.createCollection(TARGET_COLLECTION, TARGET_COLLECTION, 1, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(target.getSolrClient());
- Thread.sleep(1000);
- targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection(TARGET_COLLECTION);
- }
-
- private void deleteSourceCollection() throws Exception {
- source.deleteAllCollections();
- }
-
- private void deleteTargetcollection() throws Exception {
- target.deleteAllCollections();
- }
-
- private void createCollections() throws Exception {
- createTargetCollection();
- createSourceCollection();
- }
-
- private void deleteCollections() throws Exception {
- deleteSourceCollection();
- deleteTargetcollection();
- }
-
-}
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java
deleted file mode 100644
index 271bf67..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java
+++ /dev/null
@@ -1,332 +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.cdcr;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.lucene.util.LuceneTestCase.Nightly;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is testing the cdcr extension to the {@link org.apache.solr.handler.ReplicationHandler} and
- * {@link org.apache.solr.handler.IndexFetcher}.
- */
-@Nightly
-public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- @Override
- public void distribSetUp() throws Exception {
- schemaString = "schema15.xml"; // we need a string id
- createTargetCollection = false; // we do not need the target cluster
- shardCount = 1; // we need only one shard
- // we need a persistent directory, otherwise the UpdateHandler will erase existing tlog files after restarting a node
- System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
- super.distribSetUp();
- }
-
- /**
- * Test the scenario where the follower is killed from the start. The replication
- * strategy should fetch all the missing tlog files from the leader.
- */
- @Test
- @ShardsFixed(num = 2)
- public void testFullReplication() throws Exception {
- List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
- followers.get(0).jetty.stop();
-
- for (int i = 0; i < 10; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 10; j < (i * 10) + 10; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- assertNumDocs(100, SOURCE_COLLECTION);
-
- // Restart the follower node to trigger Replication strategy
- this.restartServer(followers.get(0));
-
- this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
- }
-
- /**
- * Test the scenario where the follower is killed before receiving all the documents. The replication
- * strategy should fetch all the missing tlog files from the leader.
- */
- @Test
- @ShardsFixed(num = 2)
- public void testPartialReplication() throws Exception {
- for (int i = 0; i < 5; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 20; j < (i * 20) + 20; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
- followers.get(0).jetty.stop();
-
- for (int i = 5; i < 10; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 20; j < (i * 20) + 20; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- assertNumDocs(200, SOURCE_COLLECTION);
-
- // Restart the follower node to trigger Replication strategy
- this.restartServer(followers.get(0));
-
- // at this stage, the follower should have replicated the 5 missing tlog files
- this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
- }
-
- /**
- * Test the scenario where the follower is killed before receiving a commit. This creates a truncated tlog
- * file on the follower node. The replication strategy should detect this truncated file, and fetch the
- * non-truncated file from the leader.
- */
- @Test
- @ShardsFixed(num = 2)
- public void testPartialReplicationWithTruncatedTlog() throws Exception {
- CloudSolrClient client = createCloudClient(SOURCE_COLLECTION);
- List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
-
- try {
- for (int i = 0; i < 10; i++) {
- for (int j = i * 20; j < (i * 20) + 20; j++) {
- client.add(getDoc(id, Integer.toString(j)));
-
- // Stop the follower in the middle of a batch to create a truncated tlog on the follower
- if (j == 45) {
- followers.get(0).jetty.stop();
- }
-
- }
- commit(SOURCE_COLLECTION);
- }
- } finally {
- client.close();
- }
-
- assertNumDocs(200, SOURCE_COLLECTION);
-
- // Restart the follower node to trigger Replication recovery
- this.restartServer(followers.get(0));
-
- // at this stage, the follower should have replicated the 5 missing tlog files
- this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
- }
-
- /**
- * Test the scenario where the follower first recovered with a PeerSync strategy, then with a Replication strategy.
- * The PeerSync strategy will generate a single tlog file for all the missing updates on the follower node.
- * If a Replication strategy occurs at a later stage, it should remove this tlog file generated by PeerSync
- * and fetch the corresponding tlog files from the leader.
- */
- @Test
- @ShardsFixed(num = 2)
- public void testPartialReplicationAfterPeerSync() throws Exception {
- for (int i = 0; i < 5; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 10; j < (i * 10) + 10; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
- followers.get(0).jetty.stop();
-
- for (int i = 5; i < 10; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 10; j < (i * 10) + 10; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- assertNumDocs(100, SOURCE_COLLECTION);
-
- // Restart the follower node to trigger PeerSync recovery
- // (the update windows between leader and follower is small enough)
- this.restartServer(followers.get(0));
-
- followers.get(0).jetty.stop();
-
- for (int i = 10; i < 15; i++) {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = i * 20; j < (i * 20) + 20; j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- }
-
- // restart the follower node to trigger Replication recovery
- this.restartServer(followers.get(0));
-
- // at this stage, the follower should have replicated the 5 missing tlog files
- this.assertUpdateLogsEquals(SOURCE_COLLECTION, 15);
- }
-
- /**
- * Test the scenario where the follower is killed while the leader is still receiving updates.
- * The follower should buffer updates while in recovery, then replay them at the end of the recovery.
- * If updates were properly buffered and replayed, then the follower should have the same number of documents
- * than the leader. This checks if cdcr tlog replication interferes with buffered updates - SOLR-8263.
- */
- @Test
- @ShardsFixed(num = 2)
- public void testReplicationWithBufferedUpdates() throws Exception {
- List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
-
- AtomicInteger numDocs = new AtomicInteger(0);
- ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-test-update-scheduler"));
- executor.scheduleWithFixedDelay(new UpdateThread(numDocs), 10, 10, TimeUnit.MILLISECONDS);
-
- // Restart the follower node to trigger Replication strategy
- this.restartServer(followers.get(0));
-
- // shutdown the update thread and wait for its completion
- executor.shutdown();
- executor.awaitTermination(500, TimeUnit.MILLISECONDS);
-
- // check that we have the expected number of documents in the cluster
- assertNumDocs(numDocs.get(), SOURCE_COLLECTION);
-
- // check that we have the expected number of documents on the follower
- assertNumDocs(numDocs.get(), followers.get(0));
- }
-
- private void assertNumDocs(int expectedNumDocs, CloudJettyRunner jetty)
- throws InterruptedException, IOException, SolrServerException {
- SolrClient client = createNewSolrServer(jetty.url);
- try {
- int cnt = 30; // timeout after 15 seconds
- AssertionError lastAssertionError = null;
- while (cnt > 0) {
- try {
- assertEquals(expectedNumDocs, client.query(new SolrQuery("*:*")).getResults().getNumFound());
- return;
- }
- catch (AssertionError e) {
- lastAssertionError = e;
- cnt--;
- Thread.sleep(500);
- }
- }
- throw new AssertionError("Timeout while trying to assert number of documents @ " + jetty.url, lastAssertionError);
- } finally {
- client.close();
- }
- }
-
- private class UpdateThread implements Runnable {
-
- private AtomicInteger numDocs;
-
- private UpdateThread(AtomicInteger numDocs) {
- this.numDocs = numDocs;
- }
-
- @Override
- public void run() {
- try {
- List<SolrInputDocument> docs = new ArrayList<>();
- for (int j = numDocs.get(); j < (numDocs.get() + 10); j++) {
- docs.add(getDoc(id, Integer.toString(j)));
- }
- index(SOURCE_COLLECTION, docs);
- numDocs.getAndAdd(10);
- if (log.isInfoEnabled()) {
- log.info("Sent batch of {} updates - numDocs:{}", docs.size(), numDocs);
- }
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- }
-
- private List<CloudJettyRunner> getShardToFollowerJetty(String collection, String shard) {
- List<CloudJettyRunner> jetties = new ArrayList<>(shardToJetty.get(collection).get(shard));
- CloudJettyRunner leader = shardToLeaderJetty.get(collection).get(shard);
- jetties.remove(leader);
- return jetties;
- }
-
- /**
- * Asserts that the update logs are in sync between the leader and follower. The leader and the followers
- * must have identical tlog files.
- */
- protected void assertUpdateLogsEquals(String collection, int numberOfTLogs) throws Exception {
- CollectionInfo info = collectInfo(collection);
- Map<String, List<CollectionInfo.CoreInfo>> shardToCoresMap = info.getShardToCoresMap();
-
- for (String shard : shardToCoresMap.keySet()) {
- Map<Long, Long> leaderFilesMeta = this.getFilesMeta(info.getLeader(shard).ulogDir);
- Map<Long, Long> followerFilesMeta = this.getFilesMeta(info.getReplicas(shard).get(0).ulogDir);
-
- assertEquals("Incorrect number of tlog files on the leader", numberOfTLogs, leaderFilesMeta.size());
- assertEquals("Incorrect number of tlog files on the follower", numberOfTLogs, followerFilesMeta.size());
-
- for (Long leaderFileVersion : leaderFilesMeta.keySet()) {
- assertTrue("Follower is missing a tlog for version " + leaderFileVersion, followerFilesMeta.containsKey(leaderFileVersion));
- assertEquals("Follower's tlog file size differs for version " + leaderFileVersion, leaderFilesMeta.get(leaderFileVersion), followerFilesMeta.get(leaderFileVersion));
- }
- }
- }
-
- private Map<Long, Long> getFilesMeta(String dir) {
- File file = new File(dir);
- if (!file.isDirectory()) {
- assertTrue("Path to tlog " + dir + " does not exists or it's not a directory.", false);
- }
-
- Map<Long, Long> filesMeta = new HashMap<>();
- for (File tlogFile : file.listFiles()) {
- filesMeta.put(Math.abs(Long.parseLong(tlogFile.getName().substring(tlogFile.getName().lastIndexOf('.') + 1))), tlogFile.length());
- }
- return filesMeta;
- }
-
-}
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrRequestHandlerTest.java
deleted file mode 100644
index 0944a61..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrRequestHandlerTest.java
+++ /dev/null
@@ -1,183 +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.cdcr;
-
-import java.util.Arrays;
-import com.google.common.collect.ImmutableMap;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.handler.CdcrParams;
-import org.junit.Test;
-
-@Nightly
-public class CdcrRequestHandlerTest extends BaseCdcrDistributedZkTest {
-
- @Override
- public void distribSetUp() throws Exception {
- schemaString = "schema15.xml"; // we need a string id
- createTargetCollection = false; // we do not need the target cluster
- super.distribSetUp();
- }
-
- // check that the life-cycle state is properly synchronised across nodes
- @Test
- @ShardsFixed(num = 2)
- public void testLifeCycleActions() throws Exception {
- // check initial status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
-
- // send start action to first shard
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
- @SuppressWarnings({"rawtypes"})
- NamedList status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- assertEquals(CdcrParams.ProcessState.STARTED.toLower(), status.get(CdcrParams.ProcessState.getParam()));
-
- // check status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STARTED, CdcrParams.BufferState.ENABLED);
-
- // Restart the leader of shard 1
- this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1));
-
- // check status - the node that died should have picked up the original state
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STARTED, CdcrParams.BufferState.ENABLED);
-
- // send stop action to second shard
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.STOP);
- status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- assertEquals(CdcrParams.ProcessState.STOPPED.toLower(), status.get(CdcrParams.ProcessState.getParam()));
-
- // check status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
- }
-
- // check the checkpoint API
- @Test
- @ShardsFixed(num = 2)
- public void testCheckpointActions() throws Exception {
- // initial request on an empty index, must return -1
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- assertEquals(-1l, rsp.get(CdcrParams.CHECKPOINT));
-
- index(SOURCE_COLLECTION, getDoc(id, "a","test_i_dvo",10)); // shard 2
-
- // only one document indexed in shard 2, the checkpoint must be still -1
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- assertEquals(-1l, rsp.get(CdcrParams.CHECKPOINT));
-
- index(SOURCE_COLLECTION, getDoc(id, "b")); // shard 1
-
- // a second document indexed in shard 1, the checkpoint must come from shard 2
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- long checkpoint1 = (Long) rsp.get(CdcrParams.CHECKPOINT);
- long expected = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertEquals(expected, checkpoint1);
-
- index(SOURCE_COLLECTION, getDoc(id, "c")); // shard 1
-
- // a third document indexed in shard 1, the checkpoint must still come from shard 2
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- assertEquals(checkpoint1, rsp.get(CdcrParams.CHECKPOINT));
-
- index(SOURCE_COLLECTION, getDoc(id, "d")); // shard 2
-
- // a fourth document indexed in shard 2, the checkpoint must come from shard 1
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- long checkpoint2 = (Long) rsp.get(CdcrParams.CHECKPOINT);
- expected = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertEquals(expected, checkpoint2);
-
- // send a delete by id
- long pre_op = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- deleteById(SOURCE_COLLECTION, Arrays.asList(new String[]{"c"})); //shard1
- // document deleted in shard1, checkpoint should come from shard2
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- long checkpoint3 = (Long) rsp.get(CdcrParams.CHECKPOINT);
- expected = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertEquals(pre_op, expected);
- assertEquals(expected, checkpoint3);
-
- // send a in-place update
- SolrInputDocument in_place_doc = new SolrInputDocument();
- in_place_doc.setField(id, "a");
- in_place_doc.setField("test_i_dvo", ImmutableMap.of("inc", 10)); //shard2
- index(SOURCE_COLLECTION, in_place_doc);
- // document updated in shard2, checkpoint should come from shard1
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- long checkpoint4 = (Long) rsp.get(CdcrParams.CHECKPOINT);
- expected = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertEquals(expected, checkpoint4);
-
- // send a delete by query
- deleteByQuery(SOURCE_COLLECTION, "*:*");
-
- // all the checkpoints must come from the DBQ
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
- long checkpoint5= (Long) rsp.get(CdcrParams.CHECKPOINT);
- assertTrue(checkpoint5 > 0); // ensure that checkpoints from deletes are in absolute form
- checkpoint5 = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertTrue(checkpoint5 > 0); // ensure that checkpoints from deletes are in absolute form
- checkpoint5 = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
- assertTrue(checkpoint5 > 0); // ensure that checkpoints from deletes are in absolute form
-
-
- // replication never started, lastProcessedVersion should be -1 for both shards
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.LASTPROCESSEDVERSION);
- long lastVersion = (Long) rsp.get(CdcrParams.LAST_PROCESSED_VERSION);
- assertEquals(-1l, lastVersion);
-
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.LASTPROCESSEDVERSION);
- lastVersion = (Long) rsp.get(CdcrParams.LAST_PROCESSED_VERSION);
- assertEquals(-1l, lastVersion);
- }
-
- // check that the buffer state is properly synchronised across nodes
- @Test
- @ShardsFixed(num = 2)
- public void testBufferActions() throws Exception {
- // check initial status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
-
- // send disable buffer action to first shard
- @SuppressWarnings({"rawtypes"})
- NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.DISABLEBUFFER);
- @SuppressWarnings({"rawtypes"})
- NamedList status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- assertEquals(CdcrParams.BufferState.DISABLED.toLower(), status.get(CdcrParams.BufferState.getParam()));
-
- // check status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.DISABLED);
-
- // Restart the leader of shard 1
- this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1));
-
- // check status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.DISABLED);
-
- // send enable buffer action to second shard
- rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.ENABLEBUFFER);
- status = (NamedList) rsp.get(CdcrParams.CdcrAction.STATUS.toLower());
- assertEquals(CdcrParams.BufferState.ENABLED.toLower(), status.get(CdcrParams.BufferState.getParam()));
-
- // check status
- this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
- }
-
-}
-
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java
deleted file mode 100644
index 869e5be..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java
+++ /dev/null
@@ -1,274 +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.cdcr;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.LinkedHashMap;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.MiniSolrCloudCluster;
-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.cloud.Slice;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.handler.CdcrParams;
-import org.apache.solr.util.TimeOut;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class CdcrTestsUtil extends SolrTestCaseJ4 {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- protected static void cdcrRestart(CloudSolrClient client) throws SolrServerException, IOException {
- cdcrStop(client);
- cdcrStart(client);
- }
-
- protected static void cdcrStart(CloudSolrClient client) throws SolrServerException, IOException {
- QueryResponse response = invokeCdcrAction(client, CdcrParams.CdcrAction.START);
- assertEquals("started", ((NamedList) response.getResponse().get("status")).get("process"));
- }
-
- protected static void cdcrStop(CloudSolrClient client) throws SolrServerException, IOException {
- QueryResponse response = invokeCdcrAction(client, CdcrParams.CdcrAction.STOP);
- assertEquals("stopped", ((NamedList) response.getResponse().get("status")).get("process"));
- }
-
- protected static void cdcrEnableBuffer(CloudSolrClient client) throws IOException, SolrServerException {
- QueryResponse response = invokeCdcrAction(client, CdcrParams.CdcrAction.ENABLEBUFFER);
- assertEquals("enabled", ((NamedList) response.getResponse().get("status")).get("buffer"));
- }
-
- protected static void cdcrDisableBuffer(CloudSolrClient client) throws IOException, SolrServerException {
- QueryResponse response = invokeCdcrAction(client, CdcrParams.CdcrAction.DISABLEBUFFER);
- assertEquals("disabled", ((NamedList) response.getResponse().get("status")).get("buffer"));
- }
-
- protected static QueryResponse invokeCdcrAction(CloudSolrClient client, CdcrParams.CdcrAction action) throws IOException, SolrServerException {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.QT, "/cdcr");
- params.set(CommonParams.ACTION, action.toLower());
- return client.query(params);
- }
-
- protected static QueryResponse getCdcrQueue(CloudSolrClient client) throws SolrServerException, IOException {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.QT, "/cdcr");
- params.set(CommonParams.ACTION, CdcrParams.QUEUES);
- return client.query(params);
- }
-
- protected static Object getFingerPrintMaxVersion(CloudSolrClient client, String shardNames, int numDocs) throws SolrServerException, IOException, InterruptedException {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CommonParams.QT, "/get");
- params.set("fingerprint", true);
- params.set("shards", shardNames);
- params.set("getVersions", numDocs);
-
- QueryResponse response = null;
- long start = System.nanoTime();
- while (System.nanoTime() - start <= TimeUnit.NANOSECONDS.convert(20, TimeUnit.SECONDS)) {
- response = client.query(params);
- if (response.getResponse() != null && response.getResponse().get("fingerprint") != null) {
- return (long) ((LinkedHashMap) response.getResponse().get("fingerprint")).get("maxVersionEncountered");
- }
- Thread.sleep(200);
- }
- log.error("maxVersionEncountered not found for client : {} in 20 attempts", client);
- return null;
- }
-
- protected static long waitForClusterToSync(long numDocs, CloudSolrClient clusterSolrClient) throws Exception {
- return waitForClusterToSync((int) numDocs, clusterSolrClient, "*:*");
- }
-
- protected static long waitForClusterToSync(int numDocs, CloudSolrClient clusterSolrClient) throws Exception {
- return waitForClusterToSync(numDocs, clusterSolrClient, "*:*");
- }
-
- protected static long waitForClusterToSync(int numDocs, CloudSolrClient clusterSolrClient, String query) throws Exception {
- long start = System.nanoTime();
- QueryResponse response = null;
- while (System.nanoTime() - start <= TimeUnit.NANOSECONDS.convert(120, TimeUnit.SECONDS)) {
- clusterSolrClient.commit();
- response = clusterSolrClient.query(new SolrQuery(query));
- if (response.getResults().getNumFound() == numDocs) {
- break;
- }
- Thread.sleep(1000);
- }
- return response != null ? response.getResults().getNumFound() : 0;
- }
-
- protected static boolean assertShardInSync(String collection, String shard, CloudSolrClient client) throws IOException, SolrServerException {
- TimeOut waitTimeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
- DocCollection docCollection = client.getZkStateReader().getClusterState().getCollection(collection);
- Slice correctSlice = null;
- for (Slice slice : docCollection.getSlices()) {
- if (shard.equals(slice.getName())) {
- correctSlice = slice;
- break;
- }
- }
- assertNotNull(correctSlice);
-
- long leaderDocCount;
- try (HttpSolrClient leaderClient = new HttpSolrClient.Builder(correctSlice.getLeader().getCoreUrl()).withHttpClient(client.getHttpClient()).build()) {
- leaderDocCount = leaderClient.query(new SolrQuery("*:*").setParam("distrib", "false")).getResults().getNumFound();
- }
-
- while (!waitTimeOut.hasTimedOut()) {
- int replicasInSync = 0;
- for (Replica replica : correctSlice.getReplicas()) {
- try (HttpSolrClient leaderClient = new HttpSolrClient.Builder(replica.getCoreUrl()).withHttpClient(client.getHttpClient()).build()) {
- long replicaDocCount = leaderClient.query(new SolrQuery("*:*").setParam("distrib", "false")).getResults().getNumFound();
- if (replicaDocCount == leaderDocCount) replicasInSync++;
- }
- }
- if (replicasInSync == correctSlice.getReplicas().size()) {
- return true;
- }
- }
- return false;
- }
-
- public static void indexRandomDocs(Integer start, Integer count, CloudSolrClient solrClient) throws Exception {
- // ADD operation on cluster 1
- int docs = 0;
- if (count == 0) {
- docs = (TEST_NIGHTLY ? 100 : 10);
- } else {
- docs = count;
- }
- for (int k = start; k < docs; k++) {
- UpdateRequest req = new UpdateRequest();
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", k);
- req.add(doc);
-
- req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
- req.process(solrClient);
- }
- }
-
- public static void indexRandomDocs(Integer count, CloudSolrClient solrClient) throws Exception {
- indexRandomDocs(0, count, solrClient);
- }
-
- public static void index(MiniSolrCloudCluster cluster, String collection, SolrInputDocument doc, boolean doCommit) throws IOException, SolrServerException {
- CloudSolrClient client = createCloudClient(cluster, collection);
- try {
- client.add(doc);
- if (doCommit) {
- client.commit(true, true);
- } else {
- client.commit(true, false);
- }
- } finally {
- client.close();
- }
- }
-
- public static void index(MiniSolrCloudCluster cluster, String collection, SolrInputDocument doc) throws IOException, SolrServerException {
- index(cluster, collection, doc, false);
- }
-
- public static CloudSolrClient createCloudClient(MiniSolrCloudCluster cluster, String defaultCollection) {
- CloudSolrClient server = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean());
- if (defaultCollection != null) server.setDefaultCollection(defaultCollection);
- return server;
- }
-
-
- public static void restartClusterNode(MiniSolrCloudCluster cluster, String collection, int index) throws Exception {
- System.setProperty("collection", collection);
- restartNode(cluster.getJettySolrRunner(index));
- System.clearProperty("collection");
- }
-
- public static void restartClusterNodes(MiniSolrCloudCluster cluster, String collection) throws Exception {
- System.setProperty("collection", collection);
- for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
- restartNode(jetty);
- }
- System.clearProperty("collection");
- }
-
- public static void restartNode(JettySolrRunner jetty) throws Exception {
- jetty.stop();
- jetty.start();
- Thread.sleep(10000);
- }
-
- public static int numberOfFiles(String dir) {
- File file = new File(dir);
- if (!file.isDirectory()) {
- assertTrue("Path to tlog " + dir + " does not exists or it's not a directory.", false);
- }
- if (log.isDebugEnabled()) {
- log.debug("Update log dir {} contains: {}", dir, file.listFiles());
- }
- return file.listFiles().length;
- }
-
- public static int getNumberOfTlogFilesOnReplicas(MiniSolrCloudCluster cluster) throws Exception {
- int count = 0;
- for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
- for (SolrCore core : jetty.getCoreContainer().getCores()) {
- count += numberOfFiles(core.getUlogDir() + "/tlog");
- }
- }
- return count;
- }
-
- public static String getNonLeaderNode(MiniSolrCloudCluster cluster, String collection) throws Exception {
- String leaderNode = getLeaderNode(cluster, collection);
- for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
- if (!jetty.getNodeName().equals(leaderNode)) {
- return jetty.getNodeName();
- }
- }
- return cluster.getJettySolrRunners().get(0).getNodeName();
- }
-
- public static String getLeaderNode(MiniSolrCloudCluster cluster, String collection) throws Exception {
- for (Replica replica : cluster.getSolrClient().getClusterStateProvider().getCollection(collection).getReplicas()) {
- if (cluster.getSolrClient().getClusterStateProvider().getCollection(collection).getLeader("shard1") == replica) {
- return replica.getNodeName();
- }
- }
- return "";
- }
-
-}
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrVersionReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrVersionReplicationTest.java
deleted file mode 100644
index 6953a32..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrVersionReplicationTest.java
+++ /dev/null
@@ -1,307 +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.cdcr;
-
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.update.processor.CdcrUpdateProcessor;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class CdcrVersionReplicationTest extends BaseCdcrDistributedZkTest {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private static final String vfield = CommonParams.VERSION_FIELD;
- SolrClient solrServer;
-
- public CdcrVersionReplicationTest() {
- schemaString = "schema15.xml"; // we need a string id
- super.createTargetCollection = false;
- }
-
- SolrClient createClientRandomly() throws Exception {
- int r = random().nextInt(100);
-
- // testing the smart cloud client (requests to leaders) is more important than testing the forwarding logic
- if (r < 80) {
- return createCloudClient(SOURCE_COLLECTION);
- }
-
- if (r < 90) {
- return createNewSolrServer(shardToJetty.get(SOURCE_COLLECTION).get(SHARD1).get(random().nextInt(2)).url);
- }
-
- return createNewSolrServer(shardToJetty.get(SOURCE_COLLECTION).get(SHARD2).get(random().nextInt(2)).url);
- }
-
- @Test
- @ShardsFixed(num = 4)
- public void testCdcrDocVersions() throws Exception {
- SolrClient client = createClientRandomly();
- try {
- handle.clear();
- handle.put("timestamp", SKIPVAL);
-
- doTestCdcrDocVersions(client);
-
- commit(SOURCE_COLLECTION); // work arround SOLR-5628
- } finally {
- client.close();
- }
- }
-
- private void doTestCdcrDocVersions(SolrClient solrClient) throws Exception {
- this.solrServer = solrClient;
-
- log.info("### STARTING doCdcrTestDocVersions - Add commands, client: {}", solrClient);
-
- vadd("doc1", 10, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "10");
- vadd("doc2", 11, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "11");
- vadd("doc3", 10, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "10");
- vadd("doc4", 11, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "11");
- commit(SOURCE_COLLECTION);
-
- // versions are preserved and verifiable both by query and by real-time get
- doQuery(solrClient, "doc1,10,doc2,11,doc3,10,doc4,11", "q", "*:*");
- doRealTimeGet("doc1,doc2,doc3,doc4", "10,11,10,11");
-
- vadd("doc1", 5, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "5");
- vadd("doc2", 10, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "10");
- vadd("doc3", 9, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "9");
- vadd("doc4", 8, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "8");
-
- // lower versions are ignored
- doRealTimeGet("doc1,doc2,doc3,doc4", "10,11,10,11");
-
- vadd("doc1", 12, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "12");
- vadd("doc2", 12, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "12");
- vadd("doc3", 12, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "12");
- vadd("doc4", 12, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "12");
-
- // higher versions are accepted
- doRealTimeGet("doc1,doc2,doc3,doc4", "12,12,12,12");
-
- // non-cdcr update requests throw a version conflict exception for non-equal versions (optimistic locking feature)
- vaddFail("doc1", 13, 409);
- vaddFail("doc2", 13, 409);
- vaddFail("doc3", 13, 409);
-
- commit(SOURCE_COLLECTION);
-
- // versions are still as they were
- doQuery(solrClient, "doc1,12,doc2,12,doc3,12,doc4,12", "q", "*:*");
-
- // query all shard replicas individually
- doQueryShardReplica(SHARD1, "doc1,12,doc2,12,doc3,12,doc4,12", "q", "*:*");
- doQueryShardReplica(SHARD2, "doc1,12,doc2,12,doc3,12,doc4,12", "q", "*:*");
-
- // optimistic locking update
- vadd("doc4", 12);
- commit(SOURCE_COLLECTION);
-
- QueryResponse rsp = solrClient.query(params("qt", "/get", "ids", "doc4"));
- long version = (long) rsp.getResults().get(0).get(vfield);
-
- // update accepted and a new version number was generated
- assertTrue(version > 1_000_000_000_000l);
-
- log.info("### STARTING doCdcrTestDocVersions - Delete commands");
-
- // send a delete update with an older version number
- vdelete("doc1", 5, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "5");
- // must ignore the delete
- doRealTimeGet("doc1", "12");
-
- // send a delete update with a higher version number
- vdelete("doc1", 13, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "13");
- // must be deleted
- doRealTimeGet("doc1", "");
-
- // send a delete update with a higher version number
- vdelete("doc4", version + 1, CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, "" + (version + 1));
- // must be deleted
- doRealTimeGet("doc4", "");
-
- commit(SOURCE_COLLECTION);
-
- // query each shard replica individually
- doQueryShardReplica(SHARD1, "doc2,12,doc3,12", "q", "*:*");
- doQueryShardReplica(SHARD2, "doc2,12,doc3,12", "q", "*:*");
-
- // version conflict thanks to optimistic locking
- if (solrClient instanceof CloudSolrClient) // TODO: it seems that optimistic locking doesn't work with forwarding, test with shard2 client
- vdeleteFail("doc2", 50, 409);
-
- // cleanup after ourselves for the next run
- // deleteByQuery should work as usual with the CDCR_UPDATE param
- doDeleteByQuery("id:doc*", CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, Long.toString(1));
- commit(SOURCE_COLLECTION);
-
- // deleteByQuery with a version lower than anything else should have no effect
- doQuery(solrClient, "doc2,12,doc3,12", "q", "*:*");
-
- doDeleteByQuery("id:doc*", CdcrUpdateProcessor.CDCR_UPDATE, "", vfield, Long.toString(51));
- commit(SOURCE_COLLECTION);
-
- // deleteByQuery with a version higher than everything else should delete all remaining docs
- doQuery(solrClient, "", "q", "*:*");
-
- // check that replicas are as expected too
- doQueryShardReplica(SHARD1, "", "q", "*:*");
- doQueryShardReplica(SHARD2, "", "q", "*:*");
- }
-
-
- // ------------------ auxiliary methods ------------------
-
-
- void doQueryShardReplica(String shard, String expectedDocs, String... queryParams) throws Exception {
- for (CloudJettyRunner jetty : shardToJetty.get(SOURCE_COLLECTION).get(shard)) {
- doQuery(jetty.client, expectedDocs, queryParams);
- }
- }
-
- void vdelete(String id, long version, String... params) throws Exception {
- UpdateRequest req = new UpdateRequest();
- req.deleteById(id);
- req.setParam(vfield, Long.toString(version));
-
- for (int i = 0; i < params.length; i += 2) {
- req.setParam(params[i], params[i + 1]);
- }
- solrServer.request(req);
- }
-
- void vdeleteFail(String id, long version, int errCode, String... params) throws Exception {
- boolean failed = false;
- try {
- vdelete(id, version, params);
- } catch (SolrException e) {
- failed = true;
- if (e.getCause() instanceof SolrException && e.getCause() != e) {
- e = (SolrException) e.getCause();
- }
- assertEquals(errCode, e.code());
- } catch (SolrServerException ex) {
- Throwable t = ex.getCause();
- if (t instanceof SolrException) {
- failed = true;
- SolrException exception = (SolrException) t;
- assertEquals(errCode, exception.code());
- }
- } catch (Exception e) {
- log.error("ERROR", e);
- }
- assertTrue(failed);
- }
-
- void vadd(String id, long version, String... params) throws Exception {
- UpdateRequest req = new UpdateRequest();
- req.add(sdoc("id", id, vfield, version));
- for (int i = 0; i < params.length; i += 2) {
- req.setParam(params[i], params[i + 1]);
- }
- solrServer.request(req);
- }
-
- void vaddFail(String id, long version, int errCode, String... params) throws Exception {
- boolean failed = false;
- try {
- vadd(id, version, params);
- } catch (SolrException e) {
- failed = true;
- if (e.getCause() instanceof SolrException && e.getCause() != e) {
- e = (SolrException) e.getCause();
- }
- assertEquals(errCode, e.code());
- } catch (SolrServerException ex) {
- Throwable t = ex.getCause();
- if (t instanceof SolrException) {
- failed = true;
- SolrException exception = (SolrException) t;
- assertEquals(errCode, exception.code());
- }
- } catch (Exception e) {
- log.error("ERROR", e);
- }
- assertTrue(failed);
- }
-
- void doQuery(SolrClient ss, String expectedDocs, String... queryParams) throws Exception {
-
- List<String> strs = StrUtils.splitSmart(expectedDocs, ",", true);
- Map<String, Object> expectedIds = new HashMap<>();
- for (int i = 0; i < strs.size(); i += 2) {
- String id = strs.get(i);
- String vS = strs.get(i + 1);
- Long v = Long.valueOf(vS);
- expectedIds.put(id, v);
- }
-
- QueryResponse rsp = ss.query(params(queryParams));
- Map<String, Object> obtainedIds = new HashMap<>();
- for (SolrDocument doc : rsp.getResults()) {
- obtainedIds.put((String) doc.get("id"), doc.get(vfield));
- }
-
- assertEquals(expectedIds, obtainedIds);
- }
-
-
- void doRealTimeGet(String ids, String versions) throws Exception {
- Map<String, Object> expectedIds = new HashMap<>();
- List<String> strs = StrUtils.splitSmart(ids, ",", true);
- List<String> verS = StrUtils.splitSmart(versions, ",", true);
- for (int i = 0; i < strs.size(); i++) {
- if (!verS.isEmpty()) {
- expectedIds.put(strs.get(i), Long.valueOf(verS.get(i)));
- }
- }
-
- QueryResponse rsp = solrServer.query(params("qt", "/get", "ids", ids));
- Map<String, Object> obtainedIds = new HashMap<>();
- for (SolrDocument doc : rsp.getResults()) {
- obtainedIds.put((String) doc.get("id"), doc.get(vfield));
- }
-
- assertEquals(expectedIds, obtainedIds);
- }
-
- void doDeleteByQuery(String q, String... reqParams) throws Exception {
- UpdateRequest req = new UpdateRequest();
- req.deleteByQuery(q);
- req.setParams(params(reqParams));
- req.process(solrServer);
- }
-
-}
-
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrWithNodesRestartsTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrWithNodesRestartsTest.java
deleted file mode 100644
index 22ebc9f..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrWithNodesRestartsTest.java
+++ /dev/null
@@ -1,359 +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.cdcr;
-
-import java.lang.invoke.MethodHandles;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.MiniSolrCloudCluster;
-import org.apache.solr.common.SolrInputDocument;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.Nightly;
-
-@Nightly // test is too long for non nightly
-public class CdcrWithNodesRestartsTest extends SolrTestCaseJ4 {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- MiniSolrCloudCluster target, source;
- CloudSolrClient sourceSolrClient, targetSolrClient;
- private static String SOURCE_COLLECTION = "cdcr-source";
- private static String TARGET_COLLECTION = "cdcr-target";
- private static String ALL_Q = "*:*";
-
- @BeforeClass
- public static void beforeClass() {
- System.clearProperty("solr.httpclient.retries");
- System.clearProperty("solr.retries.on.forward");
- System.clearProperty("solr.retries.to.followers");
- }
-
- @Before
- public void before() throws Exception {
- target = new MiniSolrCloudCluster(2, createTempDir(TARGET_COLLECTION), buildJettyConfig("/solr"));
- System.setProperty("cdcr.target.zkHost", target.getZkServer().getZkAddress());
- source = new MiniSolrCloudCluster(2, createTempDir(SOURCE_COLLECTION), buildJettyConfig("/solr"));
- }
-
- @After
- public void after() throws Exception {
- if (null != target) {
- target.shutdown();
- target = null;
- }
- if (null != source) {
- source.shutdown();
- source = null;
- }
- }
-
- @Test
- public void testBufferOnNonLeader() throws Exception {
- createCollections();
- CdcrTestsUtil.cdcrDisableBuffer(sourceSolrClient);
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
- Thread.sleep(2000);
-
- // index 100 docs
- for (int i = 0; i < 100; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // restart all the nodes at source cluster, one by one
- CdcrTestsUtil.restartClusterNodes(source, SOURCE_COLLECTION);
-
- //verify cdcr has replicated docs
- QueryResponse response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 100, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 100, CdcrTestsUtil.waitForClusterToSync(100, targetSolrClient));
- CdcrTestsUtil.assertShardInSync(SOURCE_COLLECTION, "shard1", sourceSolrClient);
- CdcrTestsUtil.assertShardInSync(TARGET_COLLECTION, "shard1", targetSolrClient);
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
- CdcrTestsUtil.cdcrStop(targetSolrClient);
-
- deleteCollections();
- }
-
- @Test
- public void testUpdateLogSynchronisation() throws Exception {
- createCollections();
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
- Thread.sleep(2000);
-
- // index 100 docs
- for (int i = 0; i < 100; i++) {
- // will perform a commit for every document and will create one tlog file per commit
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc, true);
- }
- Thread.sleep(2000);
-
- //verify cdcr has replicated docs
- QueryResponse response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 100, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 100, CdcrTestsUtil.waitForClusterToSync(100, targetSolrClient));
-
- // Get the number of tlog files on the replicas (should be equal to the number of documents indexed)
- int nTlogs = CdcrTestsUtil.getNumberOfTlogFilesOnReplicas(source);
-
- // Disable the buffer - ulog synch should start on non-leader nodes
- CdcrTestsUtil.cdcrDisableBuffer(sourceSolrClient);
- Thread.sleep(2000);
-
- int cnt = 15; // timeout after 15 seconds
- int n = 0;
- while (cnt > 0) {
- // Index a new document with a commit to trigger update log cleaning
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + random().nextLong());
- CdcrTestsUtil.index(source, "cdcr-source", doc, true);
-
- // Check the update logs on non-leader nodes, the number of tlog files should decrease
- n = CdcrTestsUtil.getNumberOfTlogFilesOnReplicas(source);
- if (n < nTlogs) {
- cnt = Integer.MIN_VALUE;
- break;
- }
- cnt--;
- Thread.sleep(1000);
- }
- if (cnt == 0) {
- throw new AssertionError("Timeout while trying to assert update logs @ source_collection, " + n + " " + nTlogs);
- }
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
- CdcrTestsUtil.cdcrStop(targetSolrClient);
-
- deleteCollections();
- }
-
- @Test
- public void testReplicationAfterRestart() throws Exception {
- createCollections();
- CdcrTestsUtil.cdcrStart(sourceSolrClient); // start CDCR
- Thread.sleep(2000);
-
- //index 100 docs
- for (int i = 0; i < 100; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- QueryResponse response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 100, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 100, CdcrTestsUtil.waitForClusterToSync(100, targetSolrClient));
- CdcrTestsUtil.assertShardInSync("cdcr-source", "shard1", sourceSolrClient);
-
- // restart all the source cluster nodes
- CdcrTestsUtil.restartClusterNodes(source, "cdcr-source");
- sourceSolrClient = source.getSolrClient();
- sourceSolrClient.setDefaultCollection("cdcr-source");
-
- // verify still the docs are there
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 100, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 100, CdcrTestsUtil.waitForClusterToSync(100, targetSolrClient));
-
- // index 100 more
- for (int i = 100; i < 200; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify still the docs are there
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 200, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 200, CdcrTestsUtil.waitForClusterToSync(200, targetSolrClient));
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
- CdcrTestsUtil.cdcrStop(targetSolrClient);
-
- deleteCollections();
- }
-
- @Test
- // commented out on: 24-Dec-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
- public void testReplicationAfterLeaderChange() throws Exception {
- createCollections();
- CdcrTestsUtil.cdcrStart(sourceSolrClient);
- Thread.sleep(2000);
-
- // index 100 docs
- for (int i = 0; i < 100; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- QueryResponse response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 100, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 100, CdcrTestsUtil.waitForClusterToSync(100, targetSolrClient));
- CdcrTestsUtil.assertShardInSync("cdcr-source", "shard1", sourceSolrClient);
-
- // restart one of the source cluster nodes
- CdcrTestsUtil.restartClusterNode(source, "cdcr-source", 0);
- sourceSolrClient = source.getSolrClient();
- sourceSolrClient.setDefaultCollection("cdcr-source");
-
- // add `100 more docs, 200 until now
- for (int i = 100; i < 200; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 200, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 200, CdcrTestsUtil.waitForClusterToSync(200, targetSolrClient));
-
- // restart the other source cluster node
- CdcrTestsUtil.restartClusterNode(source, "cdcr-source", 1);
- sourceSolrClient = source.getSolrClient();
- sourceSolrClient.setDefaultCollection("cdcr-source");
-
- // add `100 more docs, 300 until now
- for (int i = 200; i < 300; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 300, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 300, CdcrTestsUtil.waitForClusterToSync(300, targetSolrClient));
-
- // add a replica to 'target' collection
- CollectionAdminRequest.addReplicaToShard(TARGET_COLLECTION, "shard1").
- setNode(CdcrTestsUtil.getNonLeaderNode(target, TARGET_COLLECTION)).process(targetSolrClient);
- Thread.sleep(2000);
-
- // restart one of the target nodes
- CdcrTestsUtil.restartClusterNode(source, "cdcr-target", 0);
- targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
-
- // add `100 more docs, 400 until now
- for (int i = 300; i < 400; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 400, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 400, CdcrTestsUtil.waitForClusterToSync(400, targetSolrClient));
-
- // restart the other target cluster node
- CdcrTestsUtil.restartClusterNode(source, "cdcr-target", 1);
- targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
-
- // add `100 more docs, 500 until now
- for (int i = 400; i < 500; i++) {
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField("id", "doc_" + i);
- CdcrTestsUtil.index(source, "cdcr-source", doc);
- sourceSolrClient.commit();
- }
- Thread.sleep(2000);
-
- // verify cdcr has replicated docs
- response = sourceSolrClient.query(new SolrQuery(ALL_Q));
- assertEquals("source docs mismatch", 500, response.getResults().getNumFound());
- assertEquals("target docs mismatch", 500, CdcrTestsUtil.waitForClusterToSync(500, targetSolrClient));
-
- CdcrTestsUtil.cdcrStop(sourceSolrClient);
- CdcrTestsUtil.cdcrStop(targetSolrClient);
-
- deleteCollections();
- }
-
- private void createSourceCollection() throws Exception {
- source.uploadConfigSet(configset("cdcr-source"), "cdcr-source");
- CollectionAdminRequest.createCollection("cdcr-source", "cdcr-source", 1, 2)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(source.getSolrClient());
- Thread.sleep(1000);
- sourceSolrClient = source.getSolrClient();
- sourceSolrClient.setDefaultCollection("cdcr-source");
- }
-
- private void createTargetCollection() throws Exception {
- target.uploadConfigSet(configset("cdcr-target"), "cdcr-target");
- CollectionAdminRequest.createCollection("cdcr-target", "cdcr-target", 1, 1)
- .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
- .process(target.getSolrClient());
- Thread.sleep(1000);
- targetSolrClient = target.getSolrClient();
- targetSolrClient.setDefaultCollection("cdcr-target");
- }
-
- private void deleteSourceCollection() throws Exception {
- source.deleteAllCollections();
- }
-
- private void deleteTargetcollection() throws Exception {
- target.deleteAllCollections();
- }
-
- private void createCollections() throws Exception {
- createTargetCollection();
- createSourceCollection();
- }
-
- private void deleteCollections() throws Exception {
- deleteSourceCollection();
- deleteTargetcollection();
- }
-
-}
diff --git a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
index b0f684e..6e1b300 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
@@ -42,7 +42,6 @@ public class TestRealTimeGet extends TestRTGBase {
@BeforeClass
public static void beforeClass() throws Exception {
- randomizeUpdateLogImpl();
initCore("solrconfig-tlog.xml","schema_latest.xml");
}
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 4e7e12e..6eac90a 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -76,7 +76,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
public void beforeTest() throws Exception {
savedFactory = System.getProperty("solr.DirectoryFactory");
System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockFSDirectoryFactory");
- randomizeUpdateLogImpl();
initCore("solrconfig-tlog.xml","schema15.xml");
// validate that the schema was not changed to an unexpected state
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
index 7e20c79..45676a0 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
@@ -51,7 +51,6 @@ public class TestStressRecovery extends TestRTGBase {
@Before
public void beforeClass() throws Exception {
- randomizeUpdateLogImpl();
initCore("solrconfig-tlog.xml","schema15.xml");
}
diff --git a/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java b/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
deleted file mode 100644
index c1a9731..0000000
--- a/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
+++ /dev/null
@@ -1,783 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.update;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.lucene.util.LuceneTestCase.Nightly;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.util.TestInjection;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.apache.solr.common.util.Utils.fromJSONString;
-
-@Nightly
-public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
-
- private static int timeout = 60; // acquire timeout in seconds. change this to a huge number when debugging to prevent threads from advancing.
-
- // TODO: fix this test to not require FSDirectory
- static String savedFactory;
-
- @BeforeClass
- public static void beforeClass() throws Exception {
- savedFactory = System.getProperty("solr.DirectoryFactory");
- System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockFSDirectoryFactory");
- initCore("solrconfig-cdcrupdatelog.xml", "schema15.xml");
- }
-
- @AfterClass
- public static void afterClass() {
- if (savedFactory == null) {
- System.clearProperty("solr.directoryFactory");
- } else {
- System.setProperty("solr.directoryFactory", savedFactory);
- }
- }
-
- private void clearCore() throws IOException {
- clearIndex();
- assertU(commit());
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
-
- h.close();
-
- String[] files = ulog.getLogList(logDir);
- for (String file : files) {
-
- File toDelete = new File(logDir, file);
- Files.delete(toDelete.toPath()); // Should we really error out here?
- }
-
- assertEquals(0, ulog.getLogList(logDir).length);
-
- createCore();
- }
-
- private void deleteByQuery(String q) throws Exception {
- deleteByQueryAndGetVersion(q, null);
- }
-
- private void addDocs(int nDocs, int start, LinkedList<Long> versions) throws Exception {
- for (int i = 0; i < nDocs; i++) {
- versions.addFirst(addAndGetVersion(sdoc("id", Integer.toString(start + i)), null));
- }
- }
-
- private static Long getVer(SolrQueryRequest req) throws Exception {
- @SuppressWarnings({"rawtypes"})
- Map rsp = (Map) fromJSONString(JQ(req));
- @SuppressWarnings({"rawtypes"})
- Map doc = null;
- if (rsp.containsKey("doc")) {
- doc = (Map) rsp.get("doc");
- } else if (rsp.containsKey("docs")) {
- @SuppressWarnings({"rawtypes"})
- List lst = (List) rsp.get("docs");
- if (lst.size() > 0) {
- doc = (Map) lst.get(0);
- }
- } else if (rsp.containsKey("response")) {
- @SuppressWarnings({"rawtypes"})
- Map responseMap = (Map) rsp.get("response");
- @SuppressWarnings({"rawtypes"})
- List lst = (List) responseMap.get("docs");
- if (lst.size() > 0) {
- doc = (Map) lst.get(0);
- }
- }
-
- if (doc == null) return null;
-
- return (Long) doc.get("_version_");
- }
-
- @Test
- public void testLogReaderNext() throws Exception {
- this.clearCore();
-
- int start = 0;
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- CdcrUpdateLog.CdcrLogReader reader = ((CdcrUpdateLog) ulog).newLogReader(); // test reader on empty updates log
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(11, start, versions);
- start += 11;
- assertU(commit());
-
- for (int i = 0; i < 10; i++) { // 10 adds
- assertNotNull(reader.next());
- }
- Object o = reader.next();
- assertNotNull(o);
-
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- int opAndFlags = (Integer) entry.get(0);
- assertEquals(UpdateLog.COMMIT, opAndFlags & UpdateLog.OPERATION_MASK);
-
- for (int i = 0; i < 11; i++) { // 11 adds
- assertNotNull(reader.next());
- }
- o = reader.next();
- assertNotNull(o);
-
- entry = (List) o;
- opAndFlags = (Integer) entry.get(0);
- assertEquals(UpdateLog.COMMIT, opAndFlags & UpdateLog.OPERATION_MASK);
-
- assertNull(reader.next());
-
- // add a new tlog after having exhausted the reader
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // the reader should pick up the new tlog
-
- for (int i = 0; i < 11; i++) { // 10 adds + 1 commit
- assertNotNull(reader.next());
- }
- assertNull(reader.next());
- }
-
- /**
- * Check the seek method of the log reader.
- */
- @Test
- public void testLogReaderSeek() throws Exception {
- this.clearCore();
-
- int start = 0;
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- CdcrUpdateLog.CdcrLogReader reader1 = ((CdcrUpdateLog) ulog).newLogReader();
- CdcrUpdateLog.CdcrLogReader reader2 = ((CdcrUpdateLog) ulog).newLogReader();
- CdcrUpdateLog.CdcrLogReader reader3 = ((CdcrUpdateLog) ulog).newLogReader();
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(11, start, versions);
- start += 11;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // Test case where target version is equal to startVersion of tlog file
- long targetVersion = getVer(req("q", "id:10"));
-
- assertTrue(reader1.seek(targetVersion));
- Object o = reader1.next();
- assertNotNull(o);
- @SuppressWarnings({"rawtypes"})
- List entry = (List) o;
- long version = (Long) entry.get(1);
-
- assertEquals(targetVersion, version);
-
- assertNotNull(reader1.next());
-
- // test case where target version is superior to startVersion of tlog file
- targetVersion = getVer(req("q", "id:26"));
-
- assertTrue(reader2.seek(targetVersion));
- o = reader2.next();
- assertNotNull(o);
- entry = (List) o;
- version = (Long) entry.get(1);
-
- assertEquals(targetVersion, version);
-
- assertNotNull(reader2.next());
-
- // test case where target version is inferior to startVersion of oldest tlog file
- targetVersion = getVer(req("q", "id:0")) - 1;
-
- assertFalse(reader3.seek(targetVersion));
- }
-
- /**
- * Check that the log reader is able to read the new tlog
- * and pick up new entries as they appear.
- */
- @Test
- public void testLogReaderNextOnNewTLog() throws Exception {
- this.clearCore();
-
- int start = 0;
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- CdcrUpdateLog.CdcrLogReader reader = ((CdcrUpdateLog) ulog).newLogReader();
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(11, start, versions);
- start += 11;
-
- for (int i = 0; i < 22; i++) { // 21 adds + 1 commit
- assertNotNull(reader.next());
- }
-
- // we should have reach the end of the new tlog
- assertNull(reader.next());
-
- addDocs(5, start, versions);
- start += 5;
-
- // the reader should now pick up the new updates
-
- for (int i = 0; i < 5; i++) { // 5 adds
- assertNotNull(reader.next());
- }
-
- assertNull(reader.next());
- }
-
- @Test
- public void testRemoveOldLogs() throws Exception {
- this.clearCore();
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
-
- int start = 0;
- int maxReq = 50;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
- assertU(commit());
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
-
- addDocs(10, start, versions);
- start += 10;
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
- assertU(commit());
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
-
- assertEquals(2, ulog.getLogList(logDir).length);
-
- // Get a cdcr log reader to initialise a log pointer
- CdcrUpdateLog.CdcrLogReader reader = ((CdcrUpdateLog) ulog).newLogReader();
-
- addDocs(105, start, versions);
- start += 105;
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
- assertU(commit());
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
-
- // the previous two tlogs should not be removed
- assertEquals(3, ulog.getLogList(logDir).length);
-
- // move the pointer past the first tlog
- for (int i = 0; i <= 11; i++) { // 10 adds + 1 commit
- assertNotNull(reader.next());
- }
-
- addDocs(10, start, versions);
- start += 10;
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
- assertU(commit());
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
-
- // the first tlog should be removed
- assertEquals(3, ulog.getLogList(logDir).length);
-
- h.close();
- createCore();
-
- ulog = h.getCore().getUpdateHandler().getUpdateLog();
-
- addDocs(105, start, versions);
- start += 105;
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
- assertU(commit());
- assertJQ(req("qt", "/get", "getVersions", "" + maxReq), "/versions==" + versions.subList(0, Math.min(maxReq, start)));
-
- // previous tlogs should be gone now
- assertEquals(1, ulog.getLogList(logDir).length);
- }
-
- /**
- * Check that the removal of old logs is taking into consideration
- * multiple log pointers. Check also that the removal takes into consideration the
- * numRecordsToKeep limit, even if the log pointers are ahead.
- */
- @Test
- public void testRemoveOldLogsMultiplePointers() throws Exception {
- this.clearCore();
-
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
- CdcrUpdateLog.CdcrLogReader reader1 = ((CdcrUpdateLog) ulog).newLogReader();
- CdcrUpdateLog.CdcrLogReader reader2 = ((CdcrUpdateLog) ulog).newLogReader();
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(105, start, versions);
- start += 105;
- assertU(commit());
-
- // the previous two tlogs should not be removed
- assertEquals(3, ulog.getLogList(logDir).length);
-
- // move the first pointer past the first tlog
- for (int i = 0; i <= 11; i++) { // 10 adds + 1 commit
- assertNotNull(reader1.next());
- }
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // the first tlog should not be removed
- assertEquals(4, ulog.getLogList(logDir).length);
-
- // move the second pointer past the first tlog
- for (int i = 0; i <= 11; i++) { // 10 adds + 1 commit
- assertNotNull(reader2.next());
- }
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // the first tlog should be removed
- assertEquals(4, ulog.getLogList(logDir).length);
-
- // exhaust the readers
- while (reader1.next() != null) {
- }
- while (reader2.next() != null) {
- }
-
- // the readers should point to the new tlog
- // now add enough documents to trigger the numRecordsToKeep limit
-
- addDocs(80, start, versions);
- start += 80;
- assertU(commit());
-
- // the update log should kept the last 3 tlogs, which sum up to 100 records
- assertEquals(3, ulog.getLogList(logDir).length);
- }
-
- /**
- * Check that the output stream of an uncapped tlog is correctly reopen
- * and that the commit is written during recovery.
- */
- @Test
- public void testClosingOutputStreamAfterLogReplay() throws Exception {
- this.clearCore();
- try {
- TestInjection.skipIndexWriterCommitOnClose = true;
- final Semaphore logReplay = new Semaphore(0);
- final Semaphore logReplayFinish = new Semaphore(0);
-
- UpdateLog.testing_logReplayHook = () -> {
- try {
- assertTrue(logReplay.tryAcquire(timeout, TimeUnit.SECONDS));
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- };
-
- UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
-
- Deque<Long> versions = new ArrayDeque<>();
- versions.addFirst(addAndGetVersion(sdoc("id", "A11"), null));
- versions.addFirst(addAndGetVersion(sdoc("id", "A12"), null));
- versions.addFirst(addAndGetVersion(sdoc("id", "A13"), null));
-
- assertJQ(req("q", "*:*"), "/response/numFound==0");
-
- assertJQ(req("qt", "/get", "getVersions", "" + versions.size()), "/versions==" + versions);
-
- h.close();
- createCore();
- // Solr should kick this off now
- // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
-
- // verify that previous close didn't do a commit
- // recovery should be blocked by our hook
- assertJQ(req("q", "*:*"), "/response/numFound==0");
-
- // unblock recovery
- logReplay.release(1000);
-
- // wait until recovery has finished
- assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
-
- assertJQ(req("q", "*:*"), "/response/numFound==3");
-
- // The transaction log should have written a commit and close its output stream
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- assertEquals(0, ulog.logs.peekLast().refcount.get());
- assertNull(ulog.logs.peekLast().channel);
-
- ulog.logs.peekLast().incref(); // reopen the output stream to check if its ends with a commit
- assertTrue(ulog.logs.peekLast().endsWithCommit());
- ulog.logs.peekLast().decref();
- } finally {
- TestInjection.skipIndexWriterCommitOnClose = false; // reset
- UpdateLog.testing_logReplayHook = null;
- UpdateLog.testing_logReplayFinishHook = null;
- }
- }
-
- /**
- * Check the buffering of the old tlogs
- */
- @Test
- public void testBuffering() throws Exception {
- this.clearCore();
-
- CdcrUpdateLog ulog = (CdcrUpdateLog) h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(105, start, versions);
- start += 105;
- assertU(commit());
-
- // the first two tlogs should have been removed
- assertEquals(1, ulog.getLogList(logDir).length);
-
- // enable buffer
- ulog.enableBuffer();
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(105, start, versions);
- start += 105;
- assertU(commit());
-
- // no tlog should have been removed
- assertEquals(4, ulog.getLogList(logDir).length);
-
- // disable buffer
- ulog.disableBuffer();
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // old tlogs should have been removed
- assertEquals(2, ulog.getLogList(logDir).length);
- }
-
-
- @Test
- public void testSubReader() throws Exception {
- this.clearCore();
-
- CdcrUpdateLog ulog = (CdcrUpdateLog) h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
- CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- assertEquals(2, ulog.getLogList(logDir).length);
-
- // start to read the first tlog
- for (int i = 0; i < 10; i++) {
- assertNotNull(reader.next());
- }
-
- // instantiate a sub reader, and finish to read the first tlog (commit operation), plus start to read the
- // second tlog (first five adds)
- CdcrUpdateLog.CdcrLogReader subReader = reader.getSubReader();
- for (int i = 0; i < 6; i++) {
- assertNotNull(subReader.next());
- }
-
- // Five adds + one commit
- assertEquals(6, subReader.getNumberOfRemainingRecords());
-
- // Generate a new tlog
- addDocs(105, start, versions);
- start += 105;
- assertU(commit());
-
- // Even if the subreader is past the first tlog, the first tlog should not have been removed
- // since the parent reader is still pointing to it
- assertEquals(3, ulog.getLogList(logDir).length);
-
- // fast forward the parent reader with the subreader
- reader.forwardSeek(subReader);
- subReader.close();
-
- // After fast forward, the parent reader should be position on the doc15
- @SuppressWarnings({"rawtypes"})
- List o = (List) reader.next();
- assertNotNull(o);
- assertTrue("Expected SolrInputDocument but got" + o.toString() ,o.get(3) instanceof SolrInputDocument);
- assertEquals("15", ((SolrInputDocument) o.get(3)).getFieldValue("id"));
-
- // Finish to read the second tlog, and start to read the third one
- for (int i = 0; i < 6; i++) {
- assertNotNull(reader.next());
- }
-
- assertEquals(105, reader.getNumberOfRemainingRecords());
-
- // Generate a new tlog to activate tlog cleaning
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- // If the parent reader was correctly fast forwarded, it should be on the third tlog, and the first two should
- // have been removed.
- assertEquals(2, ulog.getLogList(logDir).length);
- }
-
- /**
- * Check that the reader is correctly reset to its last position
- */
- @Test
- public void testResetToLastPosition() throws Exception {
- this.clearCore();
-
- CdcrUpdateLog ulog = (CdcrUpdateLog) h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
- CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- assertEquals(2, ulog.getLogList(logDir).length);
-
- for (int i = 0; i < 22; i++) {
- Object o = reader.next();
- assertNotNull(o);
- // reset to last position
- reader.resetToLastPosition();
- // we should read the same update operation, i.e., same version number
- assertEquals(((List) o).get(1), ((List) reader.next()).get(1));
- }
- assertNull(reader.next());
- }
-
- /**
- * Check that the reader is correctly reset to its last position
- */
- @Test
- public void testGetNumberOfRemainingRecords() throws Exception {
- try {
- TestInjection.skipIndexWriterCommitOnClose = true;
- final Semaphore logReplayFinish = new Semaphore(0);
- UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
-
- this.clearCore();
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- addDocs(10, start, versions);
- start += 10;
-
- h.close();
- logReplayFinish.drainPermits();
- createCore();
-
- // At this stage, we have re-opened a capped tlog, and an uncapped tlog.
- // check that the number of remaining records is correctly computed in these two cases
-
- CdcrUpdateLog ulog = (CdcrUpdateLog) h.getCore().getUpdateHandler().getUpdateLog();
- CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
-
- // wait for the replay to finish
- assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
-
- // 20 records + 2 commits
- assertEquals(22, reader.getNumberOfRemainingRecords());
-
- for (int i = 0; i < 22; i++) {
- Object o = reader.next();
- assertNotNull(o);
- assertEquals(22 - (i + 1), reader.getNumberOfRemainingRecords());
- }
- assertNull(reader.next());
- assertEquals(0, reader.getNumberOfRemainingRecords());
-
- // It should pick up the new tlog files
- addDocs(10, start, versions);
- assertEquals(10, reader.getNumberOfRemainingRecords());
- } finally {
- TestInjection.skipIndexWriterCommitOnClose = false; // reset
- UpdateLog.testing_logReplayFinishHook = null;
- }
- }
-
- /**
- * Check that the initialisation of the log reader is picking up the tlog file that is currently being
- * written.
- */
- @Test
- public void testLogReaderInitOnNewTlog() throws Exception {
- this.clearCore();
-
- int start = 0;
-
- // Start to index some documents to instantiate the new tlog
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
-
- // Create the reader after the instantiation of the new tlog
- UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- CdcrUpdateLog.CdcrLogReader reader = ((CdcrUpdateLog) ulog).newLogReader();
-
- // Continue to index documents and commits
- addDocs(11, start, versions);
- start += 11;
- assertU(commit());
-
- // check that the log reader was initialised with the new tlog
- for (int i = 0; i < 22; i++) { // 21 adds + 1 commit
- assertNotNull(reader.next());
- }
-
- // we should have reach the end of the new tlog
- assertNull(reader.next());
- }
-
- /**
- * Check that the absolute version number is used for the update log index and for the last entry read
- */
- @Test
- public void testAbsoluteLastVersion() throws Exception {
- this.clearCore();
-
- CdcrUpdateLog ulog = (CdcrUpdateLog) h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
- CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
-
- int start = 0;
-
- LinkedList<Long> versions = new LinkedList<>();
- addDocs(10, start, versions);
- start += 10;
- deleteByQuery("*:*");
- assertU(commit());
-
- deleteByQuery("*:*");
- addDocs(10, start, versions);
- start += 10;
- assertU(commit());
-
- assertEquals(2, ulog.getLogList(logDir).length);
-
- for (long version : ulog.getStartingVersions()) {
- assertTrue(version > 0);
- }
-
- for (int i = 0; i < 10; i++) {
- reader.next();
- }
-
- // first delete
- Object o = reader.next();
- assertTrue((Long) ((List) o).get(1) < 0);
- assertTrue(reader.getLastVersion() > 0);
-
- reader.next(); // commit
... 1099 lines suppressed ...