mirror of https://github.com/apache/lucene.git
SOLR-6273: Cross Data Center Replication
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1681186 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
df514f0cd0
commit
375899fdbd
|
@ -59,9 +59,37 @@ Other Changes
|
|||
|
||||
* SOLR-7355: Switch from Google's ConcurrentLinkedHashMap to Caffeine. Only
|
||||
affects HDFS support. (Ben Manes via Shawn Heisey)
|
||||
|
||||
|
||||
================== 5.3.0 ==================
|
||||
(No Changes)
|
||||
Versions of Major Components
|
||||
---------------------
|
||||
(no changes)
|
||||
|
||||
Upgrading from Solr 5.2
|
||||
-----------------------
|
||||
(no changes)
|
||||
|
||||
Detailed Change List
|
||||
----------------------
|
||||
|
||||
New Features
|
||||
----------------------
|
||||
|
||||
* SOLR-6273: Cross Data Center Replication. Active/passive replication for separate
|
||||
SolrClouds hosted on separate data centers. (Renaud Delbru, Yonik Seeley via Erick Erickson)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
(no changes)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
(no changes)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
(no changes)
|
||||
|
||||
================== 5.2.0 ==================
|
||||
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.update.CdcrUpdateLog;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrBufferManager.class);
|
||||
|
||||
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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,171 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.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()}.
|
||||
*/
|
||||
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;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrBufferStateManager.class);
|
||||
|
||||
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.getCoreDescriptor().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.getCoreDescriptor().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.getCoreDescriptor().getCoreContainer().getZkController().getZkClient();
|
||||
try {
|
||||
if (!zkClient.exists(this.getZnodePath(), true)) {
|
||||
if (!zkClient.exists(this.getZnodeBase(), true)) {
|
||||
zkClient.makePath(this.getZnodeBase(), CreateMode.PERSISTENT, true);
|
||||
}
|
||||
zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
|
||||
log.info("Created znode {}", this.getZnodePath());
|
||||
}
|
||||
} 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);
|
||||
if (Event.EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
SolrZkClient zkClient = core.getCoreDescriptor().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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,158 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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>
|
||||
*/
|
||||
class CdcrLeaderStateManager extends CdcrStateManager {
|
||||
|
||||
private boolean amILeader = false;
|
||||
|
||||
private LeaderStateWatcher wrappedWatcher;
|
||||
private Watcher watcher;
|
||||
|
||||
private SolrCore core;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrProcessStateManager.class);
|
||||
|
||||
CdcrLeaderStateManager(final SolrCore core) {
|
||||
this.core = core;
|
||||
|
||||
// Fetch leader state and register the watcher at startup
|
||||
try {
|
||||
SolrZkClient zkClient = core.getCoreDescriptor().getCoreContainer().getZkController().getZkClient();
|
||||
ClusterState clusterState = core.getCoreDescriptor().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.getCoreDescriptor().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;
|
||||
}
|
||||
|
||||
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);
|
||||
if (Event.EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
log.info("Received new leader state @ {}:{}", collectionName, shard);
|
||||
SolrZkClient zkClient = core.getCoreDescriptor().getCoreContainer().getZkController().getZkClient();
|
||||
ClusterState clusterState = core.getCoreDescriptor().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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,249 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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";
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
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";
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,170 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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>
|
||||
*/
|
||||
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;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrProcessStateManager.class);
|
||||
|
||||
CdcrProcessStateManager(final SolrCore core) {
|
||||
this.core = core;
|
||||
|
||||
// Ensure that the status znode exists
|
||||
this.createStateNode();
|
||||
|
||||
// Register the watcher at startup
|
||||
try {
|
||||
SolrZkClient zkClient = core.getCoreDescriptor().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.getCoreDescriptor().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.getCoreDescriptor().getCoreContainer().getZkController().getZkClient();
|
||||
try {
|
||||
if (!zkClient.exists(this.getZnodePath(), true)) {
|
||||
if (!zkClient.exists(this.getZnodeBase(), true)) {
|
||||
zkClient.makePath(this.getZnodeBase(), CreateMode.PERSISTENT, true);
|
||||
}
|
||||
zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
|
||||
log.info("Created znode {}", this.getZnodePath());
|
||||
}
|
||||
} 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);
|
||||
if (Event.EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
SolrZkClient zkClient = core.getCoreDescriptor().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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,222 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.apache.solr.update.processor.DistributedUpdateProcessor;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
public class CdcrReplicator implements Runnable {
|
||||
|
||||
private final CdcrReplicatorState state;
|
||||
private final int batchSize;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrReplicator.class);
|
||||
|
||||
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 benchmakr 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 (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);
|
||||
|
||||
log.debug("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();
|
||||
}
|
||||
|
||||
private boolean isDelete(Object o) {
|
||||
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 {}. Got response {}", req, rsp);
|
||||
state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
|
||||
} else if (e instanceof CloudSolrClient.RouteException) {
|
||||
log.warn("Failed to forward update request", e);
|
||||
state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
|
||||
} else {
|
||||
log.warn("Failed to forward update request", e);
|
||||
state.reportError(CdcrReplicatorState.ErrorType.INTERNAL);
|
||||
}
|
||||
}
|
||||
|
||||
private UpdateRequest processUpdate(Object o, UpdateRequest req) {
|
||||
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
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(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version));
|
||||
return req;
|
||||
}
|
||||
|
||||
case UpdateLog.DELETE_BY_QUERY: {
|
||||
String query = (String) entry.get(2);
|
||||
req.deleteByQuery(query);
|
||||
req.setParam(DistributedUpdateProcessor.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 class CdcrReplicatorException extends Exception {
|
||||
|
||||
private final UpdateRequest req;
|
||||
private final UpdateResponse rsp;
|
||||
|
||||
public CdcrReplicatorException(UpdateRequest req, UpdateResponse rsp) {
|
||||
this.req = req;
|
||||
this.rsp = rsp;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,170 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.request.QueryRequest;
|
||||
import org.apache.solr.common.SolrException;
|
||||
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.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
|
||||
|
||||
private List<CdcrReplicatorState> replicatorStates;
|
||||
|
||||
private final CdcrReplicatorScheduler scheduler;
|
||||
private CdcrProcessStateManager processStateManager;
|
||||
private CdcrLeaderStateManager leaderStateManager;
|
||||
|
||||
private SolrCore core;
|
||||
private String path;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrReplicatorManager.class);
|
||||
|
||||
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 CloudSolrClient(zkHost, true);
|
||||
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)) {
|
||||
this.initLogReaders();
|
||||
this.scheduler.start();
|
||||
return;
|
||||
}
|
||||
|
||||
this.scheduler.shutdown();
|
||||
this.closeLogReaders();
|
||||
}
|
||||
|
||||
List<CdcrReplicatorState> getReplicatorStates() {
|
||||
return replicatorStates;
|
||||
}
|
||||
|
||||
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);
|
||||
log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
|
||||
checkpoint, collectionName, shard);
|
||||
CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
|
||||
reader.seek(checkpoint);
|
||||
state.init(reader);
|
||||
} 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());
|
||||
|
||||
SolrRequest request = new QueryRequest(params);
|
||||
request.setPath(path);
|
||||
|
||||
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();
|
||||
for (CdcrReplicatorState state : replicatorStates) {
|
||||
state.shutdown();
|
||||
}
|
||||
replicatorStates.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,118 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
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;
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrReplicatorScheduler.class);
|
||||
|
||||
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 DefaultSolrThreadFactory("cdcr-scheduler"));
|
||||
//replicatorsPool = Executors.newFixedThreadPool(poolSize, new DefaultSolrThreadFactory("cdcr-replicator"));
|
||||
replicatorsPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("cdcr-replicator"));
|
||||
|
||||
// the scheduler thread is executed every second and submits one replication task
|
||||
// per available state in the queue
|
||||
scheduler.scheduleWithFixedDelay(new Runnable() {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
int nCandidates = statesQueue.size();
|
||||
for (int i = 0; i < nCandidates; i++) {
|
||||
// a thread that pool one state from the queue, execute the replication task, and push back
|
||||
// the state in the queue when the task is completed
|
||||
replicatorsPool.execute(new Runnable() {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
CdcrReplicatorState state = statesQueue.poll();
|
||||
try {
|
||||
new CdcrReplicator(state, batchSize).run();
|
||||
} finally {
|
||||
statesQueue.offer(state);
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}, 0, timeSchedule, TimeUnit.MILLISECONDS);
|
||||
isStarted = true;
|
||||
}
|
||||
}
|
||||
|
||||
void shutdown() {
|
||||
if (isStarted) {
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,270 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.schema.TrieDateField;
|
||||
import org.apache.solr.update.CdcrUpdateLog;
|
||||
import org.apache.solr.update.UpdateLog;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
* 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 Logger log = LoggerFactory.getLogger(CdcrReplicatorState.class);
|
||||
|
||||
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.getMessage());
|
||||
}
|
||||
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, System.currentTimeMillis()));
|
||||
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[]{TrieDateField.formatExternal(new Date(entry.timestamp)), 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 TrieDateField.formatExternal(new Date(logReader.getLastVersion() >> 20));
|
||||
}
|
||||
return new String();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the benchmark timer.
|
||||
*/
|
||||
BenchmarkTimer getBenchmarkTimer() {
|
||||
return this.benchmarkTimer;
|
||||
}
|
||||
|
||||
enum ErrorType {
|
||||
INTERNAL,
|
||||
BAD_REQUEST;
|
||||
|
||||
public String toLower() {
|
||||
return toString().toLowerCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
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:
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
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 class ErrorQueueEntry {
|
||||
|
||||
private ErrorType type;
|
||||
private long timestamp;
|
||||
|
||||
private ErrorQueueEntry(ErrorType type, long timestamp) {
|
||||
this.type = type;
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
}
|
||||
|
||||
private 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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,615 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.SolrException;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
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.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
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.UpdateLog;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
/**
|
||||
* <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>
|
||||
*/
|
||||
public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAware {
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrRequestHandler.class);
|
||||
|
||||
private SolrCore core;
|
||||
private String collection;
|
||||
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(NamedList args) {
|
||||
super.init(args);
|
||||
|
||||
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 = SolrParams.toSolrParams((NamedList) updateLogSynchonizerParam);
|
||||
}
|
||||
|
||||
// Configuration of the Replicator
|
||||
Object replicatorParam = args.get(CdcrParams.REPLICATOR_PARAM);
|
||||
if (replicatorParam != null && replicatorParam instanceof NamedList) {
|
||||
replicatorConfiguration = SolrParams.toSolrParams((NamedList) replicatorParam);
|
||||
}
|
||||
|
||||
// Configuration of the Buffer
|
||||
Object bufferParam = args.get(CdcrParams.BUFFER_PARAM);
|
||||
if (bufferParam != null && bufferParam instanceof NamedList) {
|
||||
bufferConfiguration = SolrParams.toSolrParams((NamedList) bufferParam);
|
||||
}
|
||||
|
||||
// Configuration of the Replicas
|
||||
replicasConfiguration = new HashMap<>();
|
||||
List replicas = args.getAll(CdcrParams.REPLICA_PARAM);
|
||||
for (Object replica : replicas) {
|
||||
if (replicas != null && replica instanceof NamedList) {
|
||||
SolrParams params = SolrParams.toSolrParams((NamedList) replica);
|
||||
if (!replicasConfiguration.containsKey(params.get(CdcrParams.SOURCE_COLLECTION_PARAM))) {
|
||||
replicasConfiguration.put(params.get(CdcrParams.SOURCE_COLLECTION_PARAM), new ArrayList<SolrParams>());
|
||||
}
|
||||
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;
|
||||
}
|
||||
default: {
|
||||
throw new RuntimeException("Unknown action: " + action);
|
||||
}
|
||||
}
|
||||
|
||||
rsp.setHttpCaching(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inform(SolrCore core) {
|
||||
this.core = core;
|
||||
collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
|
||||
|
||||
// Make sure that the core is ZKAware
|
||||
if (!core.getCoreDescriptor().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) {
|
||||
String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
|
||||
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
|
||||
log.info("Solr core is being closed - shutting down CDCR handler @ {}:{}", collectionName, 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());
|
||||
}
|
||||
|
||||
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.getCoreDescriptor().getCoreContainer().getZkController();
|
||||
try {
|
||||
zkController.getZkStateReader().updateClusterState(true);
|
||||
} catch (Exception e) {
|
||||
log.warn("Error when updating cluster state", e);
|
||||
}
|
||||
ClusterState cstate = zkController.getClusterState();
|
||||
Collection<Slice> shards = cstate.getActiveSlices(collection);
|
||||
|
||||
ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new DefaultSolrThreadFactory("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();
|
||||
UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates();
|
||||
List<Long> versions = recentUpdates.getVersions(1);
|
||||
long lastVersion = versions.isEmpty() ? -1 : Math.abs(versions.get(0));
|
||||
rsp.add(CdcrParams.CHECKPOINT, lastVersion);
|
||||
recentUpdates.close();
|
||||
}
|
||||
|
||||
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.info("Returning the lowest last processed version {} @ {}:{}", lastProcessedVersion, collectionName, shard);
|
||||
rsp.add(CdcrParams.LAST_PROCESSED_VERSION, lastProcessedVersion);
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Manage Cross Data Center 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 {
|
||||
HttpSolrClient server = new HttpSolrClient(baseUrl);
|
||||
try {
|
||||
server.setConnectionTimeout(15000);
|
||||
server.setSoTimeout(60000);
|
||||
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
params.set(CommonParams.ACTION, CdcrParams.CdcrAction.SHARDCHECKPOINT.toString());
|
||||
|
||||
SolrRequest request = new QueryRequest(params);
|
||||
request.setPath(cdcrPath);
|
||||
|
||||
NamedList response = server.request(request);
|
||||
return (Long) response.get(CdcrParams.CHECKPOINT);
|
||||
} finally {
|
||||
server.close();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
static interface CdcrStateObserver {
|
||||
|
||||
public void stateUpdate();
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,183 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.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.util.DefaultSolrThreadFactory;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* <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
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrUpdateLogSynchronizer.class);
|
||||
|
||||
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 DefaultSolrThreadFactory("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) {
|
||||
scheduler.shutdownNow();
|
||||
scheduler = null;
|
||||
}
|
||||
}
|
||||
|
||||
private class UpdateLogSynchronisation implements Runnable {
|
||||
|
||||
private String getLeaderUrl() {
|
||||
ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
|
||||
ClusterState cstate = zkController.getClusterState();
|
||||
ZkNodeProps leaderProps = cstate.getLeader(collection, 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(leaderUrl);
|
||||
server.setConnectionTimeout(15000);
|
||||
server.setSoTimeout(60000);
|
||||
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
params.set(CommonParams.ACTION, CdcrParams.CdcrAction.LASTPROCESSEDVERSION.toString());
|
||||
|
||||
SolrRequest request = new QueryRequest(params);
|
||||
request.setPath(path);
|
||||
|
||||
long lastVersion;
|
||||
try {
|
||||
NamedList response = server.request(request);
|
||||
lastVersion = (Long) response.get(CdcrParams.LAST_PROCESSED_VERSION);
|
||||
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.getMessage());
|
||||
return;
|
||||
} finally {
|
||||
try {
|
||||
server.close();
|
||||
} catch (IOException ioe) {
|
||||
log.warn("Caught exception trying to close server: ", leaderUrl, ioe.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
// 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.getMessage());
|
||||
} catch (IOException e) {
|
||||
log.warn("Couldn't advance replica buffering tlog reader to {} (to remove old tlogs): {}", lastVersion, e.getMessage());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
log.warn("Caught unexpected exception", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -29,7 +29,6 @@ import java.nio.channels.FileChannel;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.nio.file.Paths;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -46,7 +45,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.zip.Adler32;
|
||||
|
@ -78,11 +76,12 @@ import org.apache.solr.core.DirectoryFactory;
|
|||
import org.apache.solr.core.DirectoryFactory.DirContext;
|
||||
import org.apache.solr.core.IndexDeletionPolicyWrapper;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.ReplicationHandler.FileInfo;
|
||||
import org.apache.solr.handler.ReplicationHandler.*;
|
||||
import org.apache.solr.request.LocalSolrQueryRequest;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.update.CommitUpdateCommand;
|
||||
import org.apache.solr.update.UpdateLog;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.solr.util.FileUtils;
|
||||
import org.apache.solr.util.PropertiesInputStream;
|
||||
|
@ -111,6 +110,8 @@ import static org.apache.solr.handler.ReplicationHandler.INTERNAL;
|
|||
import static org.apache.solr.handler.ReplicationHandler.MASTER_URL;
|
||||
import static org.apache.solr.handler.ReplicationHandler.OFFSET;
|
||||
import static org.apache.solr.handler.ReplicationHandler.SIZE;
|
||||
import static org.apache.solr.handler.ReplicationHandler.TLOG_FILE;
|
||||
import static org.apache.solr.handler.ReplicationHandler.TLOG_FILES;
|
||||
|
||||
/**
|
||||
* <p> Provides functionality of downloading changed index files as well as config files and a timer for scheduling fetches from the
|
||||
|
@ -138,14 +139,18 @@ 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;
|
||||
|
||||
|
||||
private volatile LocalFsFileFetcher localFileFetcher;
|
||||
|
||||
private volatile ExecutorService fsyncService;
|
||||
|
@ -180,7 +185,7 @@ public class IndexFetcher {
|
|||
LOG.warn("'masterUrl' must be specified without the /replication suffix");
|
||||
}
|
||||
this.masterUrl = masterUrl;
|
||||
|
||||
|
||||
this.replicationHandler = handler;
|
||||
String compress = (String) initArgs.get(COMPRESSION);
|
||||
useInternal = INTERNAL.equals(compress);
|
||||
|
@ -207,7 +212,7 @@ public class IndexFetcher {
|
|||
try (HttpSolrClient client = new HttpSolrClient(masterUrl, myHttpClient)) {
|
||||
client.setSoTimeout(60000);
|
||||
client.setConnectionTimeout(15000);
|
||||
|
||||
|
||||
return client.request(req);
|
||||
} catch (SolrServerException e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, e.getMessage(), e);
|
||||
|
@ -243,6 +248,10 @@ public class IndexFetcher {
|
|||
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);
|
||||
}
|
||||
|
@ -251,18 +260,18 @@ public class IndexFetcher {
|
|||
boolean fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException {
|
||||
return fetchLatestIndex(forceReplication, false);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* This command downloads all the necessary files from master to install a index commit point. Only changed files are
|
||||
* downloaded. It also downloads the conf files (if they are modified).
|
||||
*
|
||||
* @param forceReplication force a replication in all cases
|
||||
* @param forceReplication force a replication in all cases
|
||||
* @param forceCoreReload force a core reload in all cases
|
||||
* @return true on success, false if slave is already in sync
|
||||
* @throws IOException if an exception occurs
|
||||
*/
|
||||
boolean fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
|
||||
|
||||
|
||||
boolean cleanupDone = false;
|
||||
boolean successfulInstall = false;
|
||||
replicationStartTime = System.currentTimeMillis();
|
||||
|
@ -271,14 +280,14 @@ public class IndexFetcher {
|
|||
Directory indexDir = null;
|
||||
String indexDirPath;
|
||||
boolean deleteTmpIdxDir = true;
|
||||
|
||||
|
||||
if (!solrCore.getSolrCoreState().getLastReplicateIndexSuccess()) {
|
||||
// if the last replication was not a success, we force a full replication
|
||||
// when we are a bit more confident we may want to try a partial replication
|
||||
// if the error is connection related or something, but we have to be careful
|
||||
forceReplication = true;
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
//get the current 'replicateable' index version in the master
|
||||
NamedList response;
|
||||
|
@ -323,12 +332,12 @@ public class IndexFetcher {
|
|||
SolrQueryRequest req = new LocalSolrQueryRequest(solrCore, new ModifiableSolrParams());
|
||||
solrCore.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
|
||||
}
|
||||
|
||||
|
||||
//there is nothing to be replicated
|
||||
successfulInstall = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
if (!forceReplication && IndexDeletionPolicyWrapper.getCommitTimestamp(commit) == latestVersion) {
|
||||
//master and slave are already in sync just return
|
||||
LOG.info("Slave in sync with master.");
|
||||
|
@ -345,6 +354,9 @@ public class IndexFetcher {
|
|||
return false;
|
||||
}
|
||||
LOG.info("Number of files in latest index in master: " + filesToDownload.size());
|
||||
if (tlogFilesToDownload != null) {
|
||||
LOG.info("Number of tlog files in master: " + tlogFilesToDownload.size());
|
||||
}
|
||||
|
||||
// Create the sync service
|
||||
fsyncService = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("fsyncService"));
|
||||
|
@ -356,11 +368,12 @@ public class IndexFetcher {
|
|||
.getCommitTimestamp(commit) >= latestVersion
|
||||
|| commit.getGeneration() >= latestGeneration || forceReplication;
|
||||
|
||||
String tmpIdxDirName = "index." + new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(new Date());
|
||||
String timestamp = new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(new Date());
|
||||
String tmpIdxDirName = "index." + timestamp;
|
||||
tmpIndex = solrCore.getDataDir() + tmpIdxDirName;
|
||||
|
||||
tmpIndexDir = solrCore.getDirectoryFactory().get(tmpIndex, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
|
||||
|
||||
|
||||
// cindex dir...
|
||||
indexDirPath = solrCore.getIndexDir();
|
||||
indexDir = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
|
||||
|
@ -372,7 +385,7 @@ public class IndexFetcher {
|
|||
if (!isFullCopyNeeded && isIndexStale(indexDir)) {
|
||||
isFullCopyNeeded = true;
|
||||
}
|
||||
|
||||
|
||||
if (!isFullCopyNeeded) {
|
||||
// a searcher might be using some flushed but not committed segments
|
||||
// because of soft commits (which open a searcher on IW's data)
|
||||
|
@ -410,13 +423,16 @@ public class IndexFetcher {
|
|||
solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(solrCore, true);
|
||||
}
|
||||
boolean reloadCore = false;
|
||||
|
||||
|
||||
try {
|
||||
LOG.info("Starting download to " + tmpIndexDir + " fullCopy="
|
||||
+ isFullCopyNeeded);
|
||||
successfulInstall = false;
|
||||
|
||||
|
||||
downloadIndexFiles(isFullCopyNeeded, indexDir, tmpIndexDir, latestGeneration);
|
||||
if (tlogFilesToDownload != null) {
|
||||
downloadTlogFiles(timestamp, latestGeneration);
|
||||
}
|
||||
LOG.info("Total time taken for download : "
|
||||
+ ((System.currentTimeMillis() - replicationStartTime) / 1000)
|
||||
+ " secs");
|
||||
|
@ -440,7 +456,7 @@ public class IndexFetcher {
|
|||
solrCore.getDirectoryFactory().remove(indexDir);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
LOG.info("Configuration files are modified, core will be reloaded");
|
||||
logReplicationTimeAndConfFiles(modifiedConfFiles,
|
||||
successfulInstall);// write to a file time of replication and
|
||||
|
@ -464,7 +480,7 @@ public class IndexFetcher {
|
|||
solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(solrCore);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// we must reload the core after we open the IW back up
|
||||
if (successfulInstall && (reloadCore || forceCoreReload)) {
|
||||
LOG.info("Reloading SolrCore {}", solrCore.getName());
|
||||
|
@ -484,10 +500,10 @@ public class IndexFetcher {
|
|||
if (isFullCopyNeeded) {
|
||||
solrCore.getUpdateHandler().newIndexWriter(isFullCopyNeeded);
|
||||
}
|
||||
|
||||
|
||||
openNewSearcherAndUpdateCommitPoint();
|
||||
}
|
||||
|
||||
|
||||
if (!isFullCopyNeeded && !forceReplication && !successfulInstall) {
|
||||
cleanup(solrCore, tmpIndexDir, indexDir, deleteTmpIdxDir, successfulInstall);
|
||||
cleanupDone = true;
|
||||
|
@ -497,7 +513,7 @@ public class IndexFetcher {
|
|||
reloadCore);
|
||||
successfulInstall = fetchLatestIndex(true, reloadCore);
|
||||
}
|
||||
|
||||
|
||||
replicationStartTime = 0;
|
||||
return successfulInstall;
|
||||
} catch (ReplicationHandlerException e) {
|
||||
|
@ -605,7 +621,7 @@ public class IndexFetcher {
|
|||
Directory dir = null;
|
||||
try {
|
||||
dir = solrCore.getDirectoryFactory().get(solrCore.getDataDir(), DirContext.META_DATA, solrCore.getSolrConfig().indexConfig.lockType);
|
||||
|
||||
|
||||
int indexCount = 1, confFilesCount = 1;
|
||||
if (props.containsKey(TIMES_INDEX_REPLICATED)) {
|
||||
indexCount = Integer.valueOf(props.getProperty(TIMES_INDEX_REPLICATED)) + 1;
|
||||
|
@ -696,7 +712,7 @@ public class IndexFetcher {
|
|||
private void openNewSearcherAndUpdateCommitPoint() throws IOException {
|
||||
SolrQueryRequest req = new LocalSolrQueryRequest(solrCore,
|
||||
new ModifiableSolrParams());
|
||||
|
||||
|
||||
RefCounted<SolrIndexSearcher> searcher = null;
|
||||
IndexCommit commitPoint;
|
||||
try {
|
||||
|
@ -719,7 +735,7 @@ public class IndexFetcher {
|
|||
|
||||
// update the commit point in replication handler
|
||||
replicationHandler.indexCommitPoint = commitPoint;
|
||||
|
||||
|
||||
}
|
||||
|
||||
private void reloadCore() {
|
||||
|
@ -756,7 +772,7 @@ public class IndexFetcher {
|
|||
}
|
||||
for (Map<String, Object> file : confFilesToDownload) {
|
||||
String saveAs = (String) (file.get(ALIAS) == null ? file.get(NAME) : file.get(ALIAS));
|
||||
localFileFetcher = new LocalFsFileFetcher(tmpconfDir, file, saveAs, true, latestGeneration);
|
||||
localFileFetcher = new LocalFsFileFetcher(tmpconfDir, file, saveAs, CONF_FILE_SHORT, latestGeneration);
|
||||
currentFile = file;
|
||||
localFileFetcher.fetchFile();
|
||||
confFilesDownloaded.add(new HashMap<>(file));
|
||||
|
@ -770,6 +786,34 @@ public class IndexFetcher {
|
|||
}
|
||||
}
|
||||
|
||||
private void downloadTlogFiles(String timestamp, long latestGeneration) throws Exception {
|
||||
UpdateLog ulog = solrCore.getUpdateHandler().getUpdateLog();
|
||||
|
||||
LOG.info("Starting download of tlog files from master: " + tlogFilesToDownload);
|
||||
tlogFilesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
|
||||
File tmpTlogDir = new File(ulog.getLogDir(), "tlog." + getDateAsStr(new Date()));
|
||||
try {
|
||||
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();
|
||||
tlogFilesDownloaded.add(new HashMap<>(file));
|
||||
}
|
||||
// this is called before copying the files to the original conf dir
|
||||
// so that if there is an exception avoid corrupting the original files.
|
||||
terminateAndWaitFsyncService();
|
||||
copyTmpTlogFiles2Tlog(tmpTlogDir, timestamp);
|
||||
} finally {
|
||||
delTree(tmpTlogDir);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Download the index files. If a new index is needed, download all the files.
|
||||
*
|
||||
|
@ -790,7 +834,7 @@ public class IndexFetcher {
|
|||
if (!compareResult.equal || downloadCompleteIndex
|
||||
|| filesToAlwaysDownloadIfNoChecksums(filename, size, compareResult)) {
|
||||
dirFileFetcher = new DirectoryFileFetcher(tmpIndexDir, file,
|
||||
(String) file.get(NAME), false, latestGeneration);
|
||||
(String) file.get(NAME), FILE, latestGeneration);
|
||||
currentFile = file;
|
||||
dirFileFetcher.fetchFile();
|
||||
filesDownloaded.add(new HashMap<>(file));
|
||||
|
@ -829,10 +873,10 @@ public class IndexFetcher {
|
|||
LOG.warn("Could not retrieve checksum from file.", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (!compareResult.checkSummed) {
|
||||
// we don't have checksums to compare
|
||||
|
||||
|
||||
if (indexFileLen == backupIndexFileLen) {
|
||||
compareResult.equal = true;
|
||||
return compareResult;
|
||||
|
@ -843,9 +887,9 @@ public class IndexFetcher {
|
|||
return compareResult;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// we have checksums to compare
|
||||
|
||||
|
||||
if (indexFileLen == backupIndexFileLen && indexFileChecksum == backupIndexFileChecksum) {
|
||||
compareResult.equal = true;
|
||||
return compareResult;
|
||||
|
@ -878,7 +922,7 @@ public class IndexFetcher {
|
|||
} catch (NoSuchFileException | FileNotFoundException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* All the files which are common between master and slave must have same size and same checksum else we assume
|
||||
|
@ -969,7 +1013,7 @@ public class IndexFetcher {
|
|||
}
|
||||
|
||||
/**
|
||||
* Make file list
|
||||
* Make file list
|
||||
*/
|
||||
private List<File> makeTmpConfDirFileList(File dir, List<File> fileList) {
|
||||
File[] files = dir.listFiles();
|
||||
|
@ -982,7 +1026,7 @@ public class IndexFetcher {
|
|||
}
|
||||
return fileList;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* The conf files are copied to the tmp dir to the conf dir. A backup of the old file is maintained
|
||||
*/
|
||||
|
@ -1021,6 +1065,30 @@ public class IndexFetcher {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The tlog files are copied from the tmp dir to the tlog dir by renaming the directory if possible.
|
||||
* A backup of the old file is maintained.
|
||||
*/
|
||||
private void copyTmpTlogFiles2Tlog(File tmpTlogDir, String timestamp) {
|
||||
File tlogDir = new File(solrCore.getUpdateHandler().getUpdateLog().getLogDir());
|
||||
File backupTlogDir = new File(tlogDir.getParent(), UpdateLog.TLOG_NAME + "." + timestamp);
|
||||
|
||||
try {
|
||||
org.apache.commons.io.FileUtils.moveDirectory(tlogDir, backupTlogDir);
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Unable to rename: " + tlogDir + " to: " + backupTlogDir, e);
|
||||
}
|
||||
|
||||
try {
|
||||
tmpTlogDir = new File(backupTlogDir, tmpTlogDir.getName());
|
||||
org.apache.commons.io.FileUtils.moveDirectory(tmpTlogDir, tlogDir);
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Unable to rename: " + tmpTlogDir + " to: " + tlogDir, e);
|
||||
}
|
||||
}
|
||||
|
||||
private String getDateAsStr(Date d) {
|
||||
return new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(d);
|
||||
}
|
||||
|
@ -1114,10 +1182,10 @@ public class IndexFetcher {
|
|||
}
|
||||
return nameVsFile.isEmpty() ? Collections.EMPTY_LIST : nameVsFile.values();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
/**
|
||||
* This simulates File.delete exception-wise, since this class has some strange behavior with it.
|
||||
* The only difference is it returns null on success, throws SecurityException on SecurityException,
|
||||
* The only difference is it returns null on success, throws SecurityException on SecurityException,
|
||||
* otherwise returns Throwable preventing deletion (instead of false), for additional information.
|
||||
*/
|
||||
static Throwable delete(File file) {
|
||||
|
@ -1130,7 +1198,7 @@ public class IndexFetcher {
|
|||
return other;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static boolean delTree(File dir) {
|
||||
try {
|
||||
org.apache.lucene.util.IOUtils.rm(dir.toPath());
|
||||
|
@ -1159,6 +1227,20 @@ 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.EMPTY_LIST : 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.EMPTY_LIST : new ArrayList<>(tmp);
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getConfFilesToDownload() {
|
||||
//make a copy first because it can be null later
|
||||
List<Map<String, Object>> tmp = confFilesToDownload;
|
||||
|
@ -1219,7 +1301,7 @@ public class IndexFetcher {
|
|||
private boolean includeChecksum = true;
|
||||
private String fileName;
|
||||
private String saveAs;
|
||||
private boolean isConf;
|
||||
private String solrParamOutput;
|
||||
private Long indexGen;
|
||||
|
||||
private long size;
|
||||
|
@ -1230,11 +1312,11 @@ public class IndexFetcher {
|
|||
private boolean aborted = false;
|
||||
|
||||
FileFetcher(FileInterface file, Map<String, Object> fileDetails, String saveAs,
|
||||
boolean isConf, long latestGen) throws IOException {
|
||||
String solrParamOutput, long latestGen) throws IOException {
|
||||
this.file = file;
|
||||
this.fileName = (String) fileDetails.get(NAME);
|
||||
this.size = (Long) fileDetails.get(SIZE);
|
||||
this.isConf = isConf;
|
||||
this.solrParamOutput = solrParamOutput;
|
||||
this.saveAs = saveAs;
|
||||
indexGen = latestGen;
|
||||
if (includeChecksum)
|
||||
|
@ -1404,12 +1486,7 @@ public class IndexFetcher {
|
|||
params.set(GENERATION, Long.toString(indexGen));
|
||||
params.set(CommonParams.QT, "/replication");
|
||||
//add the version to download. This is used to reserve the download
|
||||
if (isConf) {
|
||||
//set cf instead of file for config file
|
||||
params.set(CONF_FILE_SHORT, fileName);
|
||||
} else {
|
||||
params.set(FILE, fileName);
|
||||
}
|
||||
params.set(solrParamOutput, fileName);
|
||||
if (useInternal) {
|
||||
params.set(COMPRESSION, "true");
|
||||
}
|
||||
|
@ -1478,8 +1555,8 @@ public class IndexFetcher {
|
|||
|
||||
private class DirectoryFileFetcher extends FileFetcher {
|
||||
DirectoryFileFetcher(Directory tmpIndexDir, Map<String, Object> fileDetails, String saveAs,
|
||||
boolean isConf, long latestGen) throws IOException {
|
||||
super(new DirectoryFile(tmpIndexDir, saveAs), fileDetails, saveAs, isConf, latestGen);
|
||||
String solrParamOutput, long latestGen) throws IOException {
|
||||
super(new DirectoryFile(tmpIndexDir, saveAs), fileDetails, saveAs, solrParamOutput, latestGen);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1527,8 +1604,8 @@ public class IndexFetcher {
|
|||
|
||||
private class LocalFsFileFetcher extends FileFetcher {
|
||||
LocalFsFileFetcher(File dir, Map<String, Object> fileDetails, String saveAs,
|
||||
boolean isConf, long latestGen) throws IOException {
|
||||
super(new LocalFsFile(dir, saveAs), fileDetails, saveAs, isConf, latestGen);
|
||||
String solrParamOutput, long latestGen) throws IOException {
|
||||
super(new LocalFsFile(dir, saveAs), fileDetails, saveAs, solrParamOutput, latestGen);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -81,7 +81,9 @@ import org.apache.solr.core.SolrEventListener;
|
|||
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.UpdateLog;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.solr.util.NumberUtils;
|
||||
import org.apache.solr.util.PropertiesInputStream;
|
||||
|
@ -121,8 +123,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
version = v;
|
||||
}
|
||||
/**
|
||||
* builds a CommitVersionInfo data for the specified IndexCommit.
|
||||
* Will never be null, ut version and generation may be zero if
|
||||
* builds a CommitVersionInfo data for the specified IndexCommit.
|
||||
* Will never be null, ut version and generation may be zero if
|
||||
* there are problems extracting them from the commit data
|
||||
*/
|
||||
public static CommitVersionInfo build(IndexCommit commit) {
|
||||
|
@ -512,8 +514,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
rawParams.set(CommonParams.WT, FILE_STREAM);
|
||||
|
||||
String cfileName = solrParams.get(CONF_FILE_SHORT);
|
||||
String tlogFileName = solrParams.get(TLOG_FILE);
|
||||
if (cfileName != null) {
|
||||
rsp.add(FILE_STREAM, new LocalFsFileStream(solrParams));
|
||||
rsp.add(FILE_STREAM, new LocalFsConfFileStream(solrParams));
|
||||
} else if (tlogFileName != null) {
|
||||
rsp.add(FILE_STREAM, new LocalFsTlogFileStream(solrParams));
|
||||
} else {
|
||||
rsp.add(FILE_STREAM, new DirectoryFileStream(solrParams));
|
||||
}
|
||||
|
@ -589,6 +594,14 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
}
|
||||
}
|
||||
rsp.add(CMD_GET_FILE_LIST, result);
|
||||
|
||||
// fetch list of tlog files only if cdcr is activated
|
||||
if (core.getUpdateHandler().getUpdateLog() != null && core.getUpdateHandler().getUpdateLog() instanceof CdcrUpdateLog) {
|
||||
List<Map<String, Object>> tlogfiles = getTlogFileList();
|
||||
LOG.info("Adding tlog files to list: " + tlogfiles);
|
||||
rsp.add(TLOG_FILES, tlogfiles);
|
||||
}
|
||||
|
||||
if (confFileNameAlias.size() < 1 || core.getCoreDescriptor().getCoreContainer().isZooKeeperAware())
|
||||
return;
|
||||
LOG.debug("Adding config files to list: " + includeConfFiles);
|
||||
|
@ -596,6 +609,19 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
rsp.add(CONF_FILES, getConfFileInfoFromCache(confFileNameAlias, confFileInfoCache));
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getTlogFileList() {
|
||||
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
|
||||
String[] logList = ulog.getLogList(new File(ulog.getLogDir()));
|
||||
List<Map<String, Object>> tlogFiles = new ArrayList<>();
|
||||
for (String fileName : logList) {
|
||||
Map<String, Object> fileMeta = new HashMap<>();
|
||||
fileMeta.put(NAME, fileName);
|
||||
fileMeta.put(SIZE, new File(ulog.getLogDir(), fileName).length());
|
||||
tlogFiles.add(fileMeta);
|
||||
}
|
||||
return tlogFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* For configuration files, checksum of the file is included because, unlike index files, they may have same content
|
||||
* but different timestamps.
|
||||
|
@ -1247,11 +1273,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
***/
|
||||
}
|
||||
if (snapshoot) {
|
||||
try {
|
||||
try {
|
||||
int numberToKeep = numberBackupsToKeep;
|
||||
if (numberToKeep < 1) {
|
||||
numberToKeep = Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
SnapShooter snapShooter = new SnapShooter(core, null, null);
|
||||
snapShooter.validateCreateSnapshot();
|
||||
snapShooter.createSnapAsync(currentCommitPoint, numberToKeep, ReplicationHandler.this);
|
||||
|
@ -1284,6 +1310,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
|
||||
protected String fileName;
|
||||
protected String cfileName;
|
||||
protected String tlogFileName;
|
||||
protected String sOffset;
|
||||
protected String sLen;
|
||||
protected String compress;
|
||||
|
@ -1304,6 +1331,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
|
||||
fileName = params.get(FILE);
|
||||
cfileName = params.get(CONF_FILE_SHORT);
|
||||
tlogFileName = params.get(TLOG_FILE);
|
||||
sOffset = params.get(OFFSET);
|
||||
sLen = params.get(LEN);
|
||||
compress = params.get(COMPRESSION);
|
||||
|
@ -1320,7 +1348,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
protected void initWrite() throws IOException {
|
||||
if (sOffset != null) offset = Long.parseLong(sOffset);
|
||||
if (sLen != null) len = Integer.parseInt(sLen);
|
||||
if (fileName == null && cfileName == null) {
|
||||
if (fileName == null && cfileName == null && tlogFileName == null) {
|
||||
// no filename do nothing
|
||||
writeNothingAndFlush();
|
||||
}
|
||||
|
@ -1370,7 +1398,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
in = dir.openInput(fileName, IOContext.READONCE);
|
||||
// if offset is mentioned move the pointer to that point
|
||||
if (offset != -1) in.seek(offset);
|
||||
|
||||
|
||||
long filelen = dir.fileLength(fileName);
|
||||
long maxBytesBeforePause = 0;
|
||||
|
||||
|
@ -1425,12 +1453,17 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
|
||||
/**This is used to write files in the conf directory.
|
||||
*/
|
||||
private class LocalFsFileStream extends DirectoryFileStream {
|
||||
private abstract class LocalFsFileStream extends DirectoryFileStream {
|
||||
|
||||
private File file;
|
||||
|
||||
public LocalFsFileStream(SolrParams solrParams) {
|
||||
super(solrParams);
|
||||
this.file = this.initFile();
|
||||
}
|
||||
|
||||
protected abstract File initFile();
|
||||
|
||||
@Override
|
||||
public void write(OutputStream out) throws IOException {
|
||||
createOutputStream(out);
|
||||
|
@ -1438,9 +1471,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
try {
|
||||
initWrite();
|
||||
|
||||
//if if is a conf file read from config directory
|
||||
File file = new File(core.getResourceLoader().getConfigDir(), cfileName);
|
||||
|
||||
if (file.exists() && file.canRead()) {
|
||||
inputStream = new FileInputStream(file);
|
||||
FileChannel channel = inputStream.getChannel();
|
||||
|
@ -1478,6 +1508,32 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
}
|
||||
}
|
||||
|
||||
private class LocalFsTlogFileStream extends LocalFsFileStream {
|
||||
|
||||
public LocalFsTlogFileStream(SolrParams solrParams) {
|
||||
super(solrParams);
|
||||
}
|
||||
|
||||
protected File initFile() {
|
||||
//if it is a tlog file read from tlog directory
|
||||
return new File(core.getUpdateHandler().getUpdateLog().getLogDir(), tlogFileName);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class LocalFsConfFileStream extends LocalFsFileStream {
|
||||
|
||||
public LocalFsConfFileStream(SolrParams solrParams) {
|
||||
super(solrParams);
|
||||
}
|
||||
|
||||
protected File initFile() {
|
||||
//if it is a conf file read from config directory
|
||||
return new File(core.getResourceLoader().getConfigDir(), cfileName);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static Integer readInterval(String interval) {
|
||||
if (interval == null)
|
||||
return null;
|
||||
|
@ -1568,6 +1624,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
|
||||
public static final String CONF_FILE_SHORT = "cf";
|
||||
|
||||
public static final String TLOG_FILE = "tlogFile";
|
||||
|
||||
public static final String CHECKSUM = "checksum";
|
||||
|
||||
public static final String ALIAS = "alias";
|
||||
|
@ -1576,6 +1634,8 @@ 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";
|
||||
|
@ -1601,15 +1661,15 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
public static final String OK_STATUS = "OK";
|
||||
|
||||
public static final String NEXT_EXECUTION_AT = "nextExecutionAt";
|
||||
|
||||
|
||||
public static final String NUMBER_BACKUPS_TO_KEEP_REQUEST_PARAM = "numberToKeep";
|
||||
|
||||
|
||||
public static final String NUMBER_BACKUPS_TO_KEEP_INIT_PARAM = "maxNumberOfBackups";
|
||||
|
||||
/**
|
||||
* Boolean param for tests that can be specified when using
|
||||
* {@link #CMD_FETCH_INDEX} to force the current request to block until
|
||||
* the fetch is complete. <b>NOTE:</b> This param is not advised for
|
||||
/**
|
||||
* Boolean param for tests that can be specified when using
|
||||
* {@link #CMD_FETCH_INDEX} to force the current request to block until
|
||||
* the fetch is complete. <b>NOTE:</b> This param is not advised for
|
||||
* non-test code, since the the duration of the fetch for non-trivial
|
||||
* indexes will likeley cause the request to time out.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,249 @@
|
|||
package org.apache.solr.update;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.channels.Channels;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.util.FastOutputStream;
|
||||
import org.apache.solr.common.util.JavaBinCodec;
|
||||
import org.apache.solr.common.util.ObjectReleaseTracker;
|
||||
|
||||
/**
|
||||
* 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, int)} and {@link #getReader(long)}.</li>
|
||||
* </ul>
|
||||
*/
|
||||
public class CdcrTransactionLog extends TransactionLog {
|
||||
|
||||
private boolean isReplaying;
|
||||
long startVersion; // (absolute) version of the first element of this transaction log
|
||||
|
||||
CdcrTransactionLog(File tlogFile, Collection<String> globalStrings) {
|
||||
super(tlogFile, globalStrings);
|
||||
|
||||
// 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)));
|
||||
|
||||
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;
|
||||
|
||||
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 writeCommit(CommitUpdateCommand cmd, int flags) {
|
||||
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 | flags); // 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
|
||||
protected 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,560 @@
|
|||
package org.apache.solr.update;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
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.solr.common.SolrException;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
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";
|
||||
|
||||
protected static Logger log = LoggerFactory.getLogger(CdcrUpdateLog.class);
|
||||
|
||||
@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];
|
||||
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.
|
||||
*/
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@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 happend 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.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) {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -75,7 +75,7 @@ public class TransactionLog {
|
|||
OutputStream os;
|
||||
FastOutputStream fos; // all accesses to this stream should be synchronized on "this" (The TransactionLog)
|
||||
int numRecords;
|
||||
|
||||
|
||||
protected volatile boolean deleteOnClose = true; // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
|
||||
|
||||
AtomicInteger refcount = new AtomicInteger(1);
|
||||
|
@ -84,7 +84,7 @@ public class TransactionLog {
|
|||
|
||||
long snapshot_size;
|
||||
int snapshot_numRecords;
|
||||
|
||||
|
||||
// write a BytesRef as a byte array
|
||||
JavaBinCodec.ObjectResolver resolver = new JavaBinCodec.ObjectResolver() {
|
||||
@Override
|
||||
|
@ -133,10 +133,8 @@ public class TransactionLog {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
TransactionLog(File tlogFile, Collection<String> globalStrings) {
|
||||
this(tlogFile, globalStrings, false);
|
||||
}
|
||||
|
@ -148,6 +146,10 @@ public class TransactionLog {
|
|||
log.debug("New TransactionLog file=" + tlogFile + ", exists=" + tlogFile.exists() + ", size=" + tlogFile.length() + ", openExisting=" + openExisting);
|
||||
}
|
||||
|
||||
// Parse tlog id from the filename
|
||||
String filename = tlogFile.getName();
|
||||
id = Long.parseLong(filename.substring(filename.indexOf('.') + 1, filename.indexOf('.') + 20));
|
||||
|
||||
this.tlogFile = tlogFile;
|
||||
raf = new RandomAccessFile(this.tlogFile, "rw");
|
||||
long start = raf.length();
|
||||
|
@ -214,7 +216,6 @@ public class TransactionLog {
|
|||
size = fos.size();
|
||||
}
|
||||
|
||||
|
||||
// the end of the file should have the end message (added during a commit) plus a 4 byte size
|
||||
byte[] buf = new byte[ END_MESSAGE.length() ];
|
||||
long pos = size - END_MESSAGE.length() - 4;
|
||||
|
@ -234,9 +235,9 @@ public class TransactionLog {
|
|||
snapshot_size = fos.size();
|
||||
snapshot_numRecords = numRecords;
|
||||
return snapshot_size;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// This could mess with any readers or reverse readers that are open, or anything that might try to do a log lookup.
|
||||
// This should only be used to roll back buffered updates, not actually applied updates.
|
||||
public void rollback(long pos) throws IOException {
|
||||
|
@ -250,7 +251,6 @@ public class TransactionLog {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public long writeData(Object o) {
|
||||
LogCodec codec = new LogCodec(resolver);
|
||||
try {
|
||||
|
@ -323,7 +323,7 @@ public class TransactionLog {
|
|||
|
||||
private void checkWriteHeader(LogCodec codec, SolrInputDocument optional) throws IOException {
|
||||
|
||||
// Unsynchronized access. We can get away with an unsynchronized access here
|
||||
// Unsynchronized access. We can get away with an unsynchronized access here
|
||||
// since we will never get a false non-zero when the position is in fact 0.
|
||||
// rollback() is the only function that can reset to zero, and it blocks updates.
|
||||
if (fos.size() != 0) return;
|
||||
|
@ -454,7 +454,7 @@ public class TransactionLog {
|
|||
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();
|
||||
|
||||
|
@ -561,7 +561,7 @@ public class TransactionLog {
|
|||
assert ObjectReleaseTracker.release(this);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void forceClose() {
|
||||
if (refcount.get() > 0) {
|
||||
log.error("Error: Forcing close of " + this);
|
||||
|
@ -595,7 +595,7 @@ public class TransactionLog {
|
|||
}
|
||||
|
||||
public class LogReader {
|
||||
private ChannelFastInputStream fis;
|
||||
protected ChannelFastInputStream fis;
|
||||
private LogCodec codec = new LogCodec(resolver);
|
||||
|
||||
public LogReader(long startingPos) {
|
||||
|
@ -614,7 +614,6 @@ public class TransactionLog {
|
|||
public Object next() throws IOException, InterruptedException {
|
||||
long pos = fis.position();
|
||||
|
||||
|
||||
synchronized (TransactionLog.this) {
|
||||
if (trace) {
|
||||
log.trace("Reading log record. pos="+pos+" currentSize="+fos.size());
|
||||
|
@ -664,7 +663,7 @@ public class TransactionLog {
|
|||
public long currentPos() {
|
||||
return fis.position();
|
||||
}
|
||||
|
||||
|
||||
// returns best effort current size
|
||||
// for info purposes
|
||||
public long currentSize() throws IOException {
|
||||
|
@ -675,8 +674,6 @@ public class TransactionLog {
|
|||
|
||||
public abstract class ReverseReader {
|
||||
|
||||
|
||||
|
||||
/** Returns the next object from the log, or null if none available.
|
||||
*
|
||||
* @return The log record, or null if EOF
|
||||
|
@ -691,9 +688,8 @@ public class TransactionLog {
|
|||
@Override
|
||||
public abstract String toString() ;
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
public class FSReverseReader extends ReverseReader {
|
||||
ChannelFastInputStream fis;
|
||||
private LogCodec codec = new LogCodec(resolver) {
|
||||
|
@ -727,7 +723,6 @@ public class TransactionLog {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/** Returns the next object from the log, or null if none available.
|
||||
*
|
||||
* @return The log record, or null if EOF
|
||||
|
@ -835,7 +830,7 @@ class ChannelFastInputStream extends FastInputStream {
|
|||
public void close() throws IOException {
|
||||
ch.close();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "readFromStream="+readFromStream +" pos="+pos +" end="+end + " bufferPos="+getBufferPos() + " position="+position() ;
|
||||
|
|
|
@ -82,8 +82,8 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
for (SolrEventListener listener : softCommitCallbacks) {
|
||||
listener.postSoftCommit();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
protected void callPostOptimizeCallbacks() {
|
||||
for (SolrEventListener listener : optimizeCallbacks) {
|
||||
listener.postCommit();
|
||||
|
@ -93,18 +93,18 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
public UpdateHandler(SolrCore core) {
|
||||
this(core, null);
|
||||
}
|
||||
|
||||
|
||||
public UpdateHandler(SolrCore core, UpdateLog updateLog) {
|
||||
this.core=core;
|
||||
idField = core.getLatestSchema().getUniqueKeyField();
|
||||
idFieldType = idField!=null ? idField.getType() : null;
|
||||
parseEventListeners();
|
||||
PluginInfo ulogPluginInfo = core.getSolrConfig().getPluginInfo(UpdateLog.class.getName());
|
||||
|
||||
|
||||
|
||||
if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled()) {
|
||||
String dataDir = (String)ulogPluginInfo.initArgs.get("dir");
|
||||
|
||||
|
||||
String ulogDir = core.getCoreDescriptor().getUlogDir();
|
||||
if (ulogDir != null) {
|
||||
dataDir = ulogDir;
|
||||
|
@ -112,7 +112,7 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
if (dataDir == null || dataDir.length()==0) {
|
||||
dataDir = core.getDataDir();
|
||||
}
|
||||
|
||||
|
||||
if (dataDir != null && dataDir.startsWith("hdfs:/")) {
|
||||
DirectoryFactory dirFactory = core.getDirectoryFactory();
|
||||
if (dirFactory instanceof HdfsDirectoryFactory) {
|
||||
|
@ -120,17 +120,18 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
} else {
|
||||
ulog = new HdfsUpdateLog();
|
||||
}
|
||||
|
||||
|
||||
} else {
|
||||
ulog = new UpdateLog();
|
||||
String className = ulogPluginInfo.className == null ? UpdateLog.class.getName() : ulogPluginInfo.className;
|
||||
ulog = core.getResourceLoader().newInstance(className, UpdateLog.class);
|
||||
}
|
||||
|
||||
|
||||
if (!core.isReloaded() && !core.getDirectoryFactory().isPersistent()) {
|
||||
ulog.clearLog(core, ulogPluginInfo);
|
||||
}
|
||||
|
||||
|
||||
log.info("Using UpdateLog implementation: " + ulog.getClass().getName());
|
||||
|
||||
|
||||
ulog.init(ulogPluginInfo);
|
||||
|
||||
ulog.init(this, core);
|
||||
|
@ -144,9 +145,9 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
/**
|
||||
* Called when the Writer should be opened again - eg when replication replaces
|
||||
* all of the index files.
|
||||
*
|
||||
*
|
||||
* @param rollback IndexWriter if true else close
|
||||
*
|
||||
*
|
||||
* @throws IOException If there is a low-level I/O error.
|
||||
*/
|
||||
public abstract void newIndexWriter(boolean rollback) throws IOException;
|
||||
|
@ -173,7 +174,7 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
{
|
||||
commitCallbacks.add( listener );
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* NOTE: this function is not thread safe. However, it is safe to call within the
|
||||
* <code>inform( SolrCore core )</code> function for <code>SolrCoreAware</code> classes.
|
||||
|
|
|
@ -17,9 +17,6 @@
|
|||
|
||||
package org.apache.solr.update;
|
||||
|
||||
import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase.FROMLEADER;
|
||||
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FilenameFilter;
|
||||
|
@ -66,6 +63,9 @@ import org.apache.solr.util.plugin.PluginInfoInitialized;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase.FROMLEADER;
|
||||
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
||||
|
||||
|
||||
/** @lucene.experimental */
|
||||
public class UpdateLog implements PluginInfoInitialized {
|
||||
|
@ -138,7 +138,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
protected Map<BytesRef,LogPtr> prevMap; // used while committing/reopening is happening
|
||||
protected Map<BytesRef,LogPtr> prevMap2; // used while committing/reopening is happening
|
||||
protected TransactionLog prevMapLog; // the transaction log used to look up entries found in prevMap
|
||||
protected TransactionLog prevMapLog2; // the transaction log used to look up entries found in prevMap
|
||||
protected TransactionLog prevMapLog2; // the transaction log used to look up entries found in prevMap2
|
||||
|
||||
protected final int numDeletesToKeep = 1000;
|
||||
protected final int numDeletesByQueryToKeep = 100;
|
||||
|
@ -281,12 +281,12 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
if (debug) {
|
||||
log.debug("UpdateHandler init: tlogDir=" + tlogDir + ", existing tlogs=" + Arrays.asList(tlogFiles) + ", next id=" + id);
|
||||
}
|
||||
|
||||
|
||||
TransactionLog oldLog = null;
|
||||
for (String oldLogName : tlogFiles) {
|
||||
File f = new File(tlogDir, oldLogName);
|
||||
try {
|
||||
oldLog = new TransactionLog( f, null, true );
|
||||
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);
|
||||
|
@ -335,11 +335,19 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns a new {@link org.apache.solr.update.TransactionLog}. Sub-classes can override this method to
|
||||
* change the implementation of the transaction log.
|
||||
*/
|
||||
public TransactionLog newTransactionLog(File tlogFile, Collection<String> globalStrings, boolean openExisting) {
|
||||
return new TransactionLog(tlogFile, globalStrings, openExisting);
|
||||
}
|
||||
|
||||
public String getLogDir() {
|
||||
return tlogDir.getAbsolutePath();
|
||||
}
|
||||
|
||||
|
||||
public List<Long> getStartingVersions() {
|
||||
return startingVersions;
|
||||
}
|
||||
|
@ -381,7 +389,6 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
logs.addFirst(oldLog);
|
||||
}
|
||||
|
||||
|
||||
public String[] getLogList(File directory) {
|
||||
final String prefix = TLOG_NAME+'.';
|
||||
String[] names = directory.list(new FilenameFilter() {
|
||||
|
@ -397,20 +404,17 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
return names;
|
||||
}
|
||||
|
||||
|
||||
public long getLastLogId() {
|
||||
if (id != -1) return id;
|
||||
if (tlogFiles.length == 0) return -1;
|
||||
String last = tlogFiles[tlogFiles.length-1];
|
||||
return Long.parseLong(last.substring(TLOG_NAME.length()+1));
|
||||
return Long.parseLong(last.substring(TLOG_NAME.length() + 1));
|
||||
}
|
||||
|
||||
|
||||
public void add(AddUpdateCommand cmd) {
|
||||
add(cmd, false);
|
||||
}
|
||||
|
||||
|
||||
public void add(AddUpdateCommand cmd, boolean clearCaches) {
|
||||
// don't log if we are replaying from another log
|
||||
// TODO: we currently need to log to maintain correct versioning, rtg, etc
|
||||
|
@ -621,7 +625,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
public boolean hasUncommittedChanges() {
|
||||
return tlog != null;
|
||||
}
|
||||
|
||||
|
||||
public void preCommit(CommitUpdateCommand cmd) {
|
||||
synchronized (this) {
|
||||
if (debug) {
|
||||
|
@ -718,13 +722,13 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
// SolrCore.verbose("TLOG: lookup: for id ",indexedId.utf8ToString(),"in map",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
if (entry == null && prevMap != null) {
|
||||
entry = prevMap.get(indexedId);
|
||||
// something found in prevMap will always be found in preMapLog (which could be tlog or prevTlog)
|
||||
// something found in prevMap will always be found in prevMapLog (which could be tlog or prevTlog)
|
||||
lookupLog = prevMapLog;
|
||||
// SolrCore.verbose("TLOG: lookup: for id ",indexedId.utf8ToString(),"in prevMap",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
}
|
||||
if (entry == null && prevMap2 != null) {
|
||||
entry = prevMap2.get(indexedId);
|
||||
// something found in prevMap2 will always be found in preMapLog2 (which could be tlog or prevTlog)
|
||||
// something found in prevMap2 will always be found in prevMapLog2 (which could be tlog or prevTlog)
|
||||
lookupLog = prevMapLog2;
|
||||
// SolrCore.verbose("TLOG: lookup: for id ",indexedId.utf8ToString(),"in prevMap2",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
}
|
||||
|
@ -758,13 +762,13 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
// SolrCore.verbose("TLOG: lookup ver: for id ",indexedId.utf8ToString(),"in map",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
if (entry == null && prevMap != null) {
|
||||
entry = prevMap.get(indexedId);
|
||||
// something found in prevMap will always be found in preMapLog (which could be tlog or prevTlog)
|
||||
// something found in prevMap will always be found in prevMapLog (which could be tlog or prevTlog)
|
||||
lookupLog = prevMapLog;
|
||||
// SolrCore.verbose("TLOG: lookup ver: for id ",indexedId.utf8ToString(),"in prevMap",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
}
|
||||
if (entry == null && prevMap2 != null) {
|
||||
entry = prevMap2.get(indexedId);
|
||||
// something found in prevMap2 will always be found in preMapLog2 (which could be tlog or prevTlog)
|
||||
// something found in prevMap2 will always be found in prevMapLog2 (which could be tlog or prevTlog)
|
||||
lookupLog = prevMapLog2;
|
||||
// SolrCore.verbose("TLOG: lookup ver: for id ",indexedId.utf8ToString(),"in prevMap2",System.identityHashCode(map),"got",entry,"lookupLog=",lookupLog);
|
||||
}
|
||||
|
@ -860,7 +864,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
protected void ensureLog() {
|
||||
if (tlog == null) {
|
||||
String newLogName = String.format(Locale.ROOT, LOG_FILENAME_PATTERN, TLOG_NAME, id);
|
||||
tlog = new TransactionLog(new File(tlogDir, newLogName), globalStrings);
|
||||
tlog = newTransactionLog(new File(tlogDir, newLogName), globalStrings, false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -879,11 +883,11 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
theLog.forceClose();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void close(boolean committed) {
|
||||
close(committed, false);
|
||||
}
|
||||
|
||||
|
||||
public void close(boolean committed, boolean deleteOnClose) {
|
||||
synchronized (this) {
|
||||
recoveryExecutor.shutdown(); // no new tasks
|
||||
|
@ -924,7 +928,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
this.id = id;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public class RecentUpdates {
|
||||
Deque<TransactionLog> logList; // newest first
|
||||
List<List<Update>> updateList;
|
||||
|
@ -935,17 +939,17 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
|
||||
public List<Long> getVersions(int n) {
|
||||
List<Long> ret = new ArrayList(n);
|
||||
|
||||
|
||||
for (List<Update> singleList : updateList) {
|
||||
for (Update ptr : singleList) {
|
||||
ret.add(ptr.version);
|
||||
if (--n <= 0) return ret;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
public Object lookup(long version) {
|
||||
Update update = updates.get(version);
|
||||
if (update == null) return null;
|
||||
|
@ -989,7 +993,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
try {
|
||||
o = reader.next();
|
||||
if (o==null) break;
|
||||
|
||||
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
List entry = (List)o;
|
||||
|
||||
|
@ -1012,13 +1016,13 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
|
||||
updatesForLog.add(update);
|
||||
updates.put(version, update);
|
||||
|
||||
|
||||
if (oper == UpdateLog.DELETE_BY_QUERY) {
|
||||
deleteByQueryList.add(update);
|
||||
} else if (oper == UpdateLog.DELETE) {
|
||||
deleteList.add(new DeleteUpdate(version, (byte[])entry.get(2)));
|
||||
}
|
||||
|
||||
|
||||
break;
|
||||
|
||||
case UpdateLog.COMMIT:
|
||||
|
@ -1048,7 +1052,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
public void close() {
|
||||
for (TransactionLog log : logList) {
|
||||
log.decref();
|
||||
|
@ -1295,7 +1299,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
|
||||
public void doReplay(TransactionLog translog) {
|
||||
try {
|
||||
loglog.warn("Starting log replay " + translog + " active="+activeLog + " starting pos=" + recoveryInfo.positionOfStart);
|
||||
loglog.warn("Starting log replay " + translog + " active=" + activeLog + " starting pos=" + recoveryInfo.positionOfStart);
|
||||
long lastStatusTime = System.nanoTime();
|
||||
tlogReader = translog.getReader(recoveryInfo.positionOfStart);
|
||||
|
||||
|
@ -1309,7 +1313,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
int operationAndFlags = 0;
|
||||
long nextCount = 0;
|
||||
|
||||
for(;;) {
|
||||
for (; ; ) {
|
||||
Object o = null;
|
||||
if (cancelApplyBufferUpdate) break;
|
||||
try {
|
||||
|
@ -1321,13 +1325,13 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
long cpos = tlogReader.currentPos();
|
||||
long csize = tlogReader.currentSize();
|
||||
loglog.info(
|
||||
"log replay status {} active={} starting pos={} current pos={} current size={} % read={}",
|
||||
translog, activeLog, recoveryInfo.positionOfStart, cpos, csize,
|
||||
Math.round(cpos / (double) csize * 100.));
|
||||
|
||||
"log replay status {} active={} starting pos={} current pos={} current size={} % read={}",
|
||||
translog, activeLog, recoveryInfo.positionOfStart, cpos, csize,
|
||||
Math.round(cpos / (double) csize * 100.));
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
o = null;
|
||||
o = tlogReader.next();
|
||||
if (o == null && activeLog) {
|
||||
|
@ -1352,7 +1356,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
SolrException.log(log,e);
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
||||
if (o == null) break;
|
||||
|
@ -1360,62 +1364,58 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
try {
|
||||
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
List entry = (List)o;
|
||||
List entry = (List) o;
|
||||
|
||||
operationAndFlags = (Integer)entry.get(0);
|
||||
operationAndFlags = (Integer) entry.get(0);
|
||||
int oper = operationAndFlags & OPERATION_MASK;
|
||||
long version = (Long) entry.get(1);
|
||||
|
||||
switch (oper) {
|
||||
case UpdateLog.ADD:
|
||||
{
|
||||
case UpdateLog.ADD: {
|
||||
recoveryInfo.adds++;
|
||||
// byte[] idBytes = (byte[]) entry.get(2);
|
||||
SolrInputDocument sdoc = (SolrInputDocument)entry.get(entry.size()-1);
|
||||
SolrInputDocument sdoc = (SolrInputDocument) entry.get(entry.size() - 1);
|
||||
AddUpdateCommand cmd = new AddUpdateCommand(req);
|
||||
// cmd.setIndexedId(new BytesRef(idBytes));
|
||||
cmd.solrDoc = sdoc;
|
||||
cmd.setVersion(version);
|
||||
cmd.setFlags(UpdateCommand.REPLAY | UpdateCommand.IGNORE_AUTOCOMMIT);
|
||||
if (debug) log.debug("add " + cmd);
|
||||
if (debug) log.debug("add " + cmd);
|
||||
|
||||
proc.processAdd(cmd);
|
||||
break;
|
||||
}
|
||||
case UpdateLog.DELETE:
|
||||
{
|
||||
case UpdateLog.DELETE: {
|
||||
recoveryInfo.deletes++;
|
||||
byte[] idBytes = (byte[]) entry.get(2);
|
||||
DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
|
||||
cmd.setIndexedId(new BytesRef(idBytes));
|
||||
cmd.setVersion(version);
|
||||
cmd.setFlags(UpdateCommand.REPLAY | UpdateCommand.IGNORE_AUTOCOMMIT);
|
||||
if (debug) log.debug("delete " + cmd);
|
||||
if (debug) log.debug("delete " + cmd);
|
||||
proc.processDelete(cmd);
|
||||
break;
|
||||
}
|
||||
|
||||
case UpdateLog.DELETE_BY_QUERY:
|
||||
{
|
||||
case UpdateLog.DELETE_BY_QUERY: {
|
||||
recoveryInfo.deleteByQuery++;
|
||||
String query = (String)entry.get(2);
|
||||
String query = (String) entry.get(2);
|
||||
DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
|
||||
cmd.query = query;
|
||||
cmd.setVersion(version);
|
||||
cmd.setFlags(UpdateCommand.REPLAY | UpdateCommand.IGNORE_AUTOCOMMIT);
|
||||
if (debug) log.debug("deleteByQuery " + cmd);
|
||||
if (debug) log.debug("deleteByQuery " + cmd);
|
||||
proc.processDelete(cmd);
|
||||
break;
|
||||
}
|
||||
|
||||
case UpdateLog.COMMIT:
|
||||
{
|
||||
case UpdateLog.COMMIT: {
|
||||
commitVersion = version;
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Operation! " + oper);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Operation! " + oper);
|
||||
}
|
||||
|
||||
if (rsp.getException() != null) {
|
||||
|
@ -1430,7 +1430,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
recoveryInfo.errors++;
|
||||
loglog.warn("REPLAY_ERR: Unexpected log entry or corrupt log. Entry=" + o, cl);
|
||||
// would be caused by a corrupt transaction log
|
||||
} catch (SolrException ex) {
|
||||
} catch (SolrException ex) {
|
||||
if (ex.code() == ErrorCode.SERVICE_UNAVAILABLE.code) {
|
||||
throw ex;
|
||||
}
|
||||
|
@ -1450,7 +1450,7 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
cmd.waitSearcher = true;
|
||||
cmd.setFlags(UpdateCommand.REPLAY);
|
||||
try {
|
||||
if (debug) log.debug("commit " + cmd);
|
||||
if (debug) log.debug("commit " + cmd);
|
||||
uhandler.commit(cmd); // this should cause a commit to be added to the incomplete log and avoid it being replayed again after a restart.
|
||||
} catch (IOException ex) {
|
||||
recoveryInfo.errors++;
|
||||
|
@ -1506,25 +1506,25 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected String getTlogDir(SolrCore core, PluginInfo info) {
|
||||
String dataDir = (String) info.initArgs.get("dir");
|
||||
|
||||
|
||||
String ulogDir = core.getCoreDescriptor().getUlogDir();
|
||||
if (ulogDir != null) {
|
||||
dataDir = ulogDir;
|
||||
}
|
||||
|
||||
|
||||
if (dataDir == null || dataDir.length() == 0) {
|
||||
dataDir = core.getDataDir();
|
||||
}
|
||||
|
||||
return dataDir + "/" + TLOG_NAME;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Clears the logs on the file system. Only call before init.
|
||||
*
|
||||
*
|
||||
* @param core the SolrCore
|
||||
* @param ulogPluginInfo the init info for the UpdateHandler
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,125 @@
|
|||
package org.apache.solr.update.processor;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
* <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>
|
||||
*/
|
||||
public class CdcrUpdateProcessor extends DistributedUpdateProcessor {
|
||||
|
||||
public static final String CDCR_UPDATE = "cdcr.update";
|
||||
|
||||
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(DistributedUpdateProcessor.VERSION_FIELD, params.get(DistributedUpdateProcessor.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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,46 @@
|
|||
package org.apache.solr.update.processor;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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
|
||||
*/
|
||||
public class CdcrUpdateProcessorFactory
|
||||
extends UpdateRequestProcessorFactory
|
||||
implements DistributingUpdateProcessorFactory {
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CdcrUpdateProcessor getInstance(SolrQueryRequest req,
|
||||
SolrQueryResponse rsp, UpdateRequestProcessor next) {
|
||||
|
||||
return new CdcrUpdateProcessor(req, rsp, next);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -964,7 +964,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
* @return whether or not to drop this cmd
|
||||
* @throws IOException If there is a low-level I/O error.
|
||||
*/
|
||||
private boolean versionAdd(AddUpdateCommand cmd) throws IOException {
|
||||
protected boolean versionAdd(AddUpdateCommand cmd) throws IOException {
|
||||
BytesRef idBytes = cmd.getIndexedId();
|
||||
|
||||
if (idBytes == null) {
|
||||
|
@ -1226,7 +1226,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
}
|
||||
}
|
||||
|
||||
private ModifiableSolrParams filterParams(SolrParams params) {
|
||||
protected ModifiableSolrParams filterParams(SolrParams params) {
|
||||
ModifiableSolrParams fparams = new ModifiableSolrParams();
|
||||
passParam(params, fparams, UpdateParams.UPDATE_CHAIN);
|
||||
passParam(params, fparams, TEST_DISTRIB_SKIP_SERVERS);
|
||||
|
@ -1333,53 +1333,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
// at this point, there is an update we need to try and apply.
|
||||
// we may or may not be the leader.
|
||||
|
||||
// Find the version
|
||||
long versionOnUpdate = cmd.getVersion();
|
||||
if (versionOnUpdate == 0) {
|
||||
String versionOnUpdateS = req.getParams().get(VERSION_FIELD);
|
||||
versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS);
|
||||
}
|
||||
versionOnUpdate = Math.abs(versionOnUpdate); // normalize to positive version
|
||||
|
||||
boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
|
||||
boolean leaderLogic = isLeader && !isReplayOrPeersync;
|
||||
|
||||
if (!leaderLogic && versionOnUpdate==0) {
|
||||
throw new SolrException(ErrorCode.BAD_REQUEST, "missing _version_ on update from leader");
|
||||
}
|
||||
|
||||
vinfo.blockUpdates();
|
||||
try {
|
||||
|
||||
if (versionsStored) {
|
||||
if (leaderLogic) {
|
||||
long version = vinfo.getNewClock();
|
||||
cmd.setVersion(-version);
|
||||
// TODO update versions in all buckets
|
||||
|
||||
doLocalDelete(cmd);
|
||||
|
||||
} else {
|
||||
cmd.setVersion(-versionOnUpdate);
|
||||
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
ulog.deleteByQuery(cmd);
|
||||
return;
|
||||
}
|
||||
|
||||
doLocalDelete(cmd);
|
||||
}
|
||||
}
|
||||
|
||||
// since we don't know which documents were deleted, the easiest thing to do is to invalidate
|
||||
// all real-time caches (i.e. UpdateLog) which involves also getting a new version of the IndexReader
|
||||
// (so cache misses will see up-to-date data)
|
||||
|
||||
} finally {
|
||||
vinfo.unblockUpdates();
|
||||
}
|
||||
versionDeleteByQuery(cmd);
|
||||
|
||||
if (zkEnabled) {
|
||||
// forward to all replicas
|
||||
|
@ -1449,6 +1406,56 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
}
|
||||
}
|
||||
|
||||
protected void versionDeleteByQuery(DeleteUpdateCommand cmd) throws IOException {
|
||||
// Find the version
|
||||
long versionOnUpdate = cmd.getVersion();
|
||||
if (versionOnUpdate == 0) {
|
||||
String versionOnUpdateS = req.getParams().get(VERSION_FIELD);
|
||||
versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS);
|
||||
}
|
||||
versionOnUpdate = Math.abs(versionOnUpdate); // normalize to positive version
|
||||
|
||||
boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
|
||||
boolean leaderLogic = isLeader && !isReplayOrPeersync;
|
||||
|
||||
if (!leaderLogic && versionOnUpdate == 0) {
|
||||
throw new SolrException(ErrorCode.BAD_REQUEST, "missing _version_ on update from leader");
|
||||
}
|
||||
|
||||
vinfo.blockUpdates();
|
||||
try {
|
||||
|
||||
if (versionsStored) {
|
||||
if (leaderLogic) {
|
||||
long version = vinfo.getNewClock();
|
||||
cmd.setVersion(-version);
|
||||
// TODO update versions in all buckets
|
||||
|
||||
doLocalDelete(cmd);
|
||||
|
||||
} else {
|
||||
cmd.setVersion(-versionOnUpdate);
|
||||
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
ulog.deleteByQuery(cmd);
|
||||
return;
|
||||
}
|
||||
|
||||
doLocalDelete(cmd);
|
||||
}
|
||||
}
|
||||
|
||||
// since we don't know which documents were deleted, the easiest thing to do is to invalidate
|
||||
// all real-time caches (i.e. UpdateLog) which involves also getting a new version of the IndexReader
|
||||
// (so cache misses will see up-to-date data)
|
||||
|
||||
} finally {
|
||||
vinfo.unblockUpdates();
|
||||
}
|
||||
}
|
||||
|
||||
// internal helper method to tell if we are the leader for an add or deleteById update
|
||||
boolean isLeader(UpdateCommand cmd) {
|
||||
updateCommand = cmd;
|
||||
|
@ -1482,7 +1489,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Cannot talk to ZooKeeper - Updates are disabled.");
|
||||
}
|
||||
|
||||
private boolean versionDelete(DeleteUpdateCommand cmd) throws IOException {
|
||||
protected boolean versionDelete(DeleteUpdateCommand cmd) throws IOException {
|
||||
|
||||
BytesRef idBytes = cmd.getIndexedId();
|
||||
|
||||
|
|
|
@ -144,7 +144,7 @@ public class SolrCLI {
|
|||
}
|
||||
|
||||
/**
|
||||
* Runs a SolrCloud tool with CloudSolrServer initialized
|
||||
* Runs a SolrCloud tool with CloudSolrClient initialized
|
||||
*/
|
||||
protected abstract int runCloudTool(CloudSolrClient cloudSolrClient, CommandLine cli)
|
||||
throws Exception;
|
||||
|
@ -1217,10 +1217,10 @@ public class SolrCLI {
|
|||
|
||||
int toolExitStatus = 0;
|
||||
|
||||
try (CloudSolrClient cloudSolrServer = new CloudSolrClient(zkHost)) {
|
||||
try (CloudSolrClient cloudSolrClient = new CloudSolrClient(zkHost)) {
|
||||
System.out.println("Connecting to ZooKeeper at " + zkHost);
|
||||
cloudSolrServer.connect();
|
||||
toolExitStatus = runCloudTool(cloudSolrServer, cli);
|
||||
cloudSolrClient.connect();
|
||||
toolExitStatus = runCloudTool(cloudSolrClient, cli);
|
||||
} catch (Exception exc) {
|
||||
// since this is a CLI, spare the user the stacktrace
|
||||
String excMsg = exc.getMessage();
|
||||
|
|
|
@ -0,0 +1,85 @@
|
|||
<?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>
|
||||
|
||||
<dataDir>${solr.data.dir:}</dataDir>
|
||||
|
||||
<xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
|
||||
|
||||
<requestHandler name="standard" class="solr.StandardRequestHandler">
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/get" class="solr.RealTimeGetHandler">
|
||||
<lst name="defaults">
|
||||
<str name="omitHeader">true</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy"/>
|
||||
|
||||
<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>
|
||||
|
||||
<requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers"/>
|
||||
|
||||
</config>
|
||||
|
|
@ -0,0 +1,59 @@
|
|||
<?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>
|
||||
|
||||
<dataDir>${solr.data.dir:}</dataDir>
|
||||
|
||||
<xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
|
||||
|
||||
<requestHandler name="standard" class="solr.StandardRequestHandler">
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/get" class="solr.RealTimeGetHandler">
|
||||
<lst name="defaults">
|
||||
<str name="omitHeader">true</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy"/>
|
||||
|
||||
<requestHandler name="/update" class="solr.UpdateRequestHandler"/>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
<updateLog class="solr.CdcrUpdateLog">
|
||||
<str name="dir">${solr.ulog.dir:}</str>
|
||||
</updateLog>
|
||||
</updateHandler>
|
||||
|
||||
<requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers"/>
|
||||
|
||||
</config>
|
|
@ -0,0 +1,812 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.http.params.CoreConnectionPNames;
|
||||
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.QueryRequest;
|
||||
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.cloud.*;
|
||||
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.NamedList;
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.CdcrParams;
|
||||
import org.apache.solr.servlet.SolrDispatchFilter;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.junit.*;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import static org.apache.solr.cloud.OverseerCollectionProcessor.*;
|
||||
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
|
||||
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
|
||||
|
||||
/**
|
||||
* <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 {
|
||||
|
||||
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,
|
||||
ZkStateReader.toJSON(Collections.singletonMap("urlScheme", "https")),
|
||||
CreateMode.PERSISTENT, 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
|
||||
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 {
|
||||
destroyServers();
|
||||
}
|
||||
|
||||
protected CloudSolrClient createCloudClient(String defaultCollection) {
|
||||
CloudSolrClient server = new CloudSolrClient(zkServer.getZkAddress(), random().nextBoolean());
|
||||
server.setParallelUpdates(random().nextBoolean());
|
||||
if (defaultCollection != null) server.setDefaultCollection(defaultCollection);
|
||||
server.getLbClient().getHttpClient().getParams()
|
||||
.setParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 30000);
|
||||
return server;
|
||||
}
|
||||
|
||||
protected void printLayout() throws Exception {
|
||||
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT);
|
||||
zkClient.printLayoutToStdOut();
|
||||
zkClient.close();
|
||||
}
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of documents in a given collection
|
||||
*/
|
||||
protected long getNumDocs(String collection) throws SolrServerException, IOException {
|
||||
CloudSolrClient client = createCloudClient(collection);
|
||||
try {
|
||||
return client.query(new SolrQuery("*:*")).getResults().getNumFound();
|
||||
} finally {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Invokes a CDCR action on a given node.
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
for (CloudJettyRunner jetty : cloudJettys.get(collection)) { // check all replicas
|
||||
NamedList rsp = invokeCdcrAction(jetty, CdcrParams.CdcrAction.STATUS);
|
||||
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.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.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<>();
|
||||
int maxShardsPerNode = 1;
|
||||
|
||||
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, maxShardsPerNode, client, sb.toString());
|
||||
} finally {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
|
||||
private CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
|
||||
String collectionName, int numShards, int replicationFactor,
|
||||
int maxShardsPerNode, SolrClient client, String createNodeSetStr)
|
||||
throws SolrServerException, IOException {
|
||||
return createCollection(collectionInfos, collectionName,
|
||||
ZkNodeProps.makeMap(
|
||||
NUM_SLICES, numShards,
|
||||
REPLICATION_FACTOR, replicationFactor,
|
||||
CREATE_NODE_SET, createNodeSetStr,
|
||||
MAX_SHARDS_PER_NODE, maxShardsPerNode),
|
||||
client, null);
|
||||
}
|
||||
|
||||
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(NUM_SLICES);
|
||||
if (numShards == null) {
|
||||
String shardNames = (String) collectionProps.get(SHARDS_PROP);
|
||||
numShards = StrUtils.splitSmart(shardNames, ',').size();
|
||||
}
|
||||
Integer replicationFactor = (Integer) collectionProps.get(REPLICATION_FACTOR);
|
||||
if (replicationFactor == null) {
|
||||
replicationFactor = (Integer) OverseerCollectionProcessor.COLL_PROPS.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);
|
||||
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 SolrServerException, IOException {
|
||||
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 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;
|
||||
ChaosMonkey.stop(jetty);
|
||||
ChaosMonkey.start(jetty);
|
||||
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";
|
||||
System.setProperty("collection", temporaryCollection);
|
||||
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);
|
||||
jettys.add(jetty);
|
||||
}
|
||||
|
||||
ZkStateReader zkStateReader = ((SolrDispatchFilter) jettys.get(0)
|
||||
.getDispatchFilter().getFilter()).getCores().getZkController()
|
||||
.getZkStateReader();
|
||||
|
||||
// now wait till we see the leader for each shard
|
||||
for (int i = 1; i <= shardCount; i++) {
|
||||
this.printLayout();
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
// delete the temporary collection - we will create our own collections later
|
||||
this.deleteCollection(temporaryCollection);
|
||||
System.clearProperty("collection");
|
||||
|
||||
return nodeNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void destroyServers() throws Exception {
|
||||
for (JettySolrRunner runner : jettys) {
|
||||
try {
|
||||
ChaosMonkey.stop(runner);
|
||||
} 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();
|
||||
zkStateReader.updateClusterState(true);
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
// we need to update the jetty's shard so that it registers itself to the right shard when restarted
|
||||
this.shard = shard;
|
||||
this.jetty.setShards(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;
|
||||
this.jetty.setCoreNodeName(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 = new HttpSolrClient(baseUrl);
|
||||
s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
|
||||
s.setDefaultMaxConnectionsPerHost(100);
|
||||
s.setMaxTotalConnections(100);
|
||||
return s;
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
protected void waitForReplicationToComplete(String collectionName, String shardId) throws Exception {
|
||||
while (true) {
|
||||
log.info("Checking queue size @ {}:{}", collectionName, shardId);
|
||||
long size = this.getQueueSize(collectionName, shardId);
|
||||
if (size <= 0) {
|
||||
return;
|
||||
}
|
||||
log.info("Waiting for replication to complete. Queue size: {} @ {}:{}", size, collectionName, shardId);
|
||||
Thread.sleep(1000); // wait a bit for the replication to complete
|
||||
}
|
||||
}
|
||||
|
||||
protected long getQueueSize(String collectionName, String shardId) throws Exception {
|
||||
NamedList rsp = this.invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.QUEUES);
|
||||
NamedList host = (NamedList) ((NamedList) rsp.get(CdcrParams.QUEUES)).getVal(0);
|
||||
NamedList status = (NamedList) host.get(TARGET_COLLECTION);
|
||||
return (Long) status.get(CdcrParams.QUEUE_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts that the number of transaction logs across all the shards
|
||||
*/
|
||||
protected void assertUpdateLogs(String collection, int maxNumberOfTLogs) throws Exception {
|
||||
CollectionInfo info = collectInfo(collection);
|
||||
Map<String, List<CollectionInfo.CoreInfo>> shardToCoresMap = info.getShardToCoresMap();
|
||||
|
||||
int leaderLogs = 0;
|
||||
ArrayList<Integer> replicasLogs = new ArrayList<>(Collections.nCopies(replicationFactor - 1, 0));
|
||||
|
||||
for (String shard : shardToCoresMap.keySet()) {
|
||||
leaderLogs += numberOfFiles(info.getLeader(shard).ulogDir);
|
||||
for (int i = 0; i < replicationFactor - 1; i++) {
|
||||
replicasLogs.set(i, replicasLogs.get(i) + numberOfFiles(info.getReplicas(shard).get(i).ulogDir));
|
||||
}
|
||||
}
|
||||
|
||||
for (Integer replicaLogs : replicasLogs) {
|
||||
log.info("Number of logs in update log on leader {} and on replica {}", leaderLogs, replicaLogs);
|
||||
|
||||
// replica logs must be always equal or superior to leader logs
|
||||
assertTrue(String.format(Locale.ENGLISH, "Number of tlogs on replica: %d is different than on leader: %d.",
|
||||
replicaLogs, leaderLogs), leaderLogs <= replicaLogs);
|
||||
|
||||
assertTrue(String.format(Locale.ENGLISH, "Number of tlogs on leader: %d is superior to: %d.",
|
||||
leaderLogs, maxNumberOfTLogs), maxNumberOfTLogs >= leaderLogs);
|
||||
|
||||
assertTrue(String.format(Locale.ENGLISH, "Number of tlogs on replica: %d is superior to: %d.",
|
||||
replicaLogs, maxNumberOfTLogs), maxNumberOfTLogs >= replicaLogs);
|
||||
}
|
||||
}
|
||||
|
||||
private 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);
|
||||
}
|
||||
log.info("Update log dir {} contains: {}", dir, file.listFiles());
|
||||
return file.listFiles().length;
|
||||
}
|
||||
|
||||
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) {
|
||||
SolrDispatchFilter filter = (SolrDispatchFilter) jettyRunner.jetty.getDispatchFilter().getFilter();
|
||||
for (SolrCore core : filter.getCores().getCores()) {
|
||||
info.addCore(core, shard, shardToLeaderJetty.get(collection).containsValue(jettyRunner));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return info;
|
||||
}
|
||||
|
||||
protected 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 class CoreInfo {
|
||||
String collectionName;
|
||||
String shard;
|
||||
boolean isLeader;
|
||||
String ulogDir;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,598 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.handler.CdcrParams;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
@Slow
|
||||
public class CdcrReplicationDistributedZkTest extends BaseCdcrDistributedZkTest {
|
||||
|
||||
@Override
|
||||
public void distribSetUp() throws Exception {
|
||||
schemaString = "schema15.xml"; // we need a string id
|
||||
super.distribSetUp();
|
||||
}
|
||||
|
||||
@Test
|
||||
@ShardsFixed(num = 4)
|
||||
public void doTest() throws Exception {
|
||||
this.doTestDeleteCreateSourceCollection();
|
||||
this.doTestTargetCollectionNotAvailable();
|
||||
this.doTestReplicationStartStop();
|
||||
this.doTestReplicationAfterRestart();
|
||||
this.doTestReplicationAfterLeaderChange();
|
||||
this.doTestUpdateLogSynchronisation();
|
||||
this.doTestBufferOnNonLeader();
|
||||
this.doTestOps();
|
||||
this.doTestBatchAddsWithDelete();
|
||||
this.doTestBatchBoundaries();
|
||||
this.doTestResilienceWithDeleteByQueryOnTarget();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that the test framework handles properly the creation and deletion of collections and the
|
||||
* restart of servers.
|
||||
*/
|
||||
public void doTestDeleteCreateSourceCollection() throws Exception {
|
||||
log.info("Indexing documents");
|
||||
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
docs.add(getDoc(id, Integer.toString(i)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
index(TARGET_COLLECTION, docs);
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Restarting leader @ source_collection:shard1");
|
||||
|
||||
this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1));
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Clearing source_collection");
|
||||
|
||||
this.clearSourceCollection();
|
||||
|
||||
assertEquals(0, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Restarting leader @ target_collection:shard1");
|
||||
|
||||
this.restartServer(shardToLeaderJetty.get(TARGET_COLLECTION).get(SHARD1));
|
||||
|
||||
assertEquals(0, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Clearing target_collection");
|
||||
|
||||
this.clearTargetCollection();
|
||||
|
||||
assertEquals(0, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(0, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
assertCollectionExpectations(SOURCE_COLLECTION);
|
||||
assertCollectionExpectations(TARGET_COLLECTION);
|
||||
}
|
||||
|
||||
public void doTestTargetCollectionNotAvailable() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// send start action to first shard
|
||||
NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
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);
|
||||
|
||||
// Kill all the servers of the target
|
||||
this.deleteCollection(TARGET_COLLECTION);
|
||||
|
||||
// Index a few documents to trigger the replication
|
||||
index(SOURCE_COLLECTION, getDoc(id, "a"));
|
||||
index(SOURCE_COLLECTION, getDoc(id, "b"));
|
||||
index(SOURCE_COLLECTION, getDoc(id, "c"));
|
||||
index(SOURCE_COLLECTION, getDoc(id, "d"));
|
||||
index(SOURCE_COLLECTION, getDoc(id, "e"));
|
||||
index(SOURCE_COLLECTION, getDoc(id, "f"));
|
||||
|
||||
assertEquals(6, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
Thread.sleep(1000); // wait a bit for the replicator thread to be triggered
|
||||
|
||||
rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.ERRORS);
|
||||
NamedList collections = (NamedList) ((NamedList) rsp.get(CdcrParams.ERRORS)).getVal(0);
|
||||
NamedList errors = (NamedList) collections.get(TARGET_COLLECTION);
|
||||
assertTrue(0 < (Long) errors.get(CdcrParams.CONSECUTIVE_ERRORS));
|
||||
NamedList lastErrors = (NamedList) errors.get(CdcrParams.LAST);
|
||||
assertNotNull(lastErrors);
|
||||
assertTrue(0 < lastErrors.size());
|
||||
}
|
||||
|
||||
public void doTestReplicationStartStop() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection(); // this might log a warning to indicate he was not able to delete the collection (collection was deleted in the previous test)
|
||||
|
||||
int start = 0;
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (; start < 10; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(0, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.STOP);
|
||||
|
||||
docs.clear();
|
||||
for (; start < 110; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
assertEquals(110, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
// Start again CDCR, the source cluster should reinitialise its log readers
|
||||
// with the latest checkpoints
|
||||
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
assertEquals(110, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(110, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that the replication manager is properly restarted after a node failure.
|
||||
*/
|
||||
public void doTestReplicationAfterRestart() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
log.info("Starting CDCR");
|
||||
|
||||
// send start action to first shard
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
log.info("Indexing 10 documents");
|
||||
|
||||
int start = 0;
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (; start < 10; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
log.info("Querying source collection");
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
log.info("Waiting for replication");
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
log.info("Querying target collection");
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Restarting shard1");
|
||||
|
||||
this.restartServers(shardToJetty.get(SOURCE_COLLECTION).get(SHARD1));
|
||||
|
||||
log.info("Indexing 100 documents");
|
||||
|
||||
docs.clear();
|
||||
for (; start < 110; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
log.info("Querying source collection");
|
||||
|
||||
assertEquals(110, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
log.info("Waiting for replication");
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
log.info("Querying target collection");
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
assertEquals(110, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that the replication manager is properly started after a change of leader.
|
||||
* This test also checks that the log readers on the new leaders are initialised with
|
||||
* the target's checkpoint.
|
||||
*/
|
||||
public void doTestReplicationAfterLeaderChange() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
log.info("Starting CDCR");
|
||||
|
||||
// send start action to first shard
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
log.info("Indexing 10 documents");
|
||||
|
||||
int start = 0;
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (; start < 10; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
log.info("Querying source collection");
|
||||
|
||||
assertEquals(10, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
log.info("Waiting for replication");
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
log.info("Querying target collection");
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
assertEquals(10, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
log.info("Restarting target leaders");
|
||||
|
||||
// Close all the leaders, then restart them
|
||||
this.restartServer(shardToLeaderJetty.get(TARGET_COLLECTION).get(SHARD1));
|
||||
this.restartServer(shardToLeaderJetty.get(TARGET_COLLECTION).get(SHARD2));
|
||||
|
||||
log.info("Restarting source leaders");
|
||||
|
||||
// Close all the leaders, then restart them
|
||||
this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1));
|
||||
this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2));
|
||||
|
||||
log.info("Checking queue size of new source leaders");
|
||||
|
||||
// If the log readers of the new leaders are initialised with the target's checkpoint, the
|
||||
// queue size must be inferior to the current number of documents indexed.
|
||||
// The queue might be not completely empty since the new target checkpoint is probably not the
|
||||
// last document received
|
||||
assertTrue(this.getQueueSize(SOURCE_COLLECTION, SHARD1) < 10);
|
||||
assertTrue(this.getQueueSize(SOURCE_COLLECTION, SHARD2) < 10);
|
||||
|
||||
log.info("Indexing 100 documents");
|
||||
|
||||
docs.clear();
|
||||
for (; start < 110; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
log.info("Querying source collection");
|
||||
|
||||
assertEquals(110, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
log.info("Waiting for replication");
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
log.info("Querying target collection");
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
assertEquals(110, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that the update logs are synchronised between leader and non-leader nodes
|
||||
*/
|
||||
public void doTestUpdateLogSynchronisation() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// buffering is enabled by default, so disable it
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.DISABLEBUFFER);
|
||||
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
for (int i = 0; i < 50; i++) {
|
||||
index(SOURCE_COLLECTION, getDoc(id, Integer.toString(i))); // will perform a commit for every document
|
||||
}
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
// Stop CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.STOP);
|
||||
|
||||
assertEquals(50, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(50, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
index(SOURCE_COLLECTION, getDoc(id, Integer.toString(0))); // trigger update log cleaning on the non-leader nodes
|
||||
|
||||
// some of the tlogs should be trimmed, we must have less than 50 tlog files on both leader and non-leader
|
||||
assertUpdateLogs(SOURCE_COLLECTION, 50);
|
||||
|
||||
for (int i = 50; i < 100; i++) {
|
||||
index(SOURCE_COLLECTION, getDoc(id, Integer.toString(i)));
|
||||
}
|
||||
|
||||
index(SOURCE_COLLECTION, getDoc(id, Integer.toString(0))); // trigger update log cleaning on the non-leader nodes
|
||||
|
||||
// at this stage, we should have created one tlog file per document, and some of them must have been cleaned on the
|
||||
// leader since we are not buffering and replication is stopped, (we should have exactly 10 tlog files on the leader
|
||||
// and 11 on the non-leader)
|
||||
// the non-leader must have synchronised its update log with its leader
|
||||
assertUpdateLogs(SOURCE_COLLECTION, 50);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that the buffer is always activated on non-leader nodes.
|
||||
*/
|
||||
public void doTestBufferOnNonLeader() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// buffering is enabled by default, so disable it
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.DISABLEBUFFER);
|
||||
|
||||
// Start CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
// Index documents
|
||||
for (int i = 0; i < 200; i++) {
|
||||
index(SOURCE_COLLECTION, getDoc(id, Integer.toString(i))); // will perform a commit for every document
|
||||
}
|
||||
|
||||
// And immediately, close all the leaders, then restart them. It is likely that the replication will not be
|
||||
// performed fully, and therefore be continued by the new leader
|
||||
// At this stage, the new leader must have been elected
|
||||
this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1));
|
||||
this.restartServer(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2));
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
// If the non-leader node were buffering updates, then the replication must be complete
|
||||
assertEquals(200, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(200, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check the ops statistics.
|
||||
*/
|
||||
public void doTestOps() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// Index documents
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (int i = 0; i < 200; i++) {
|
||||
docs.add(getDoc(id, Integer.toString(i)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// Start CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
NamedList rsp = this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.OPS);
|
||||
NamedList collections = (NamedList) ((NamedList) rsp.get(CdcrParams.OPERATIONS_PER_SECOND)).getVal(0);
|
||||
NamedList ops = (NamedList) collections.get(TARGET_COLLECTION);
|
||||
double opsAll = (Double) ops.get(CdcrParams.COUNTER_ALL);
|
||||
double opsAdds = (Double) ops.get(CdcrParams.COUNTER_ADDS);
|
||||
assertTrue(opsAll > 0);
|
||||
assertEquals(opsAll, opsAdds, 0);
|
||||
|
||||
double opsDeletes = (Double) ops.get(CdcrParams.COUNTER_DELETES);
|
||||
assertEquals(0, opsDeletes, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that batch updates with deletes
|
||||
*/
|
||||
public void doTestBatchAddsWithDelete() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// Index 50 documents
|
||||
int start = 0;
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (; start < 50; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// Delete 10 documents: 10-19
|
||||
List<String> ids = new ArrayList<>();
|
||||
for (int id = 10; id < 20; id++) {
|
||||
ids.add(Integer.toString(id));
|
||||
}
|
||||
deleteById(SOURCE_COLLECTION, ids);
|
||||
|
||||
// Index 10 documents
|
||||
docs = new ArrayList<>();
|
||||
for (; start < 60; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// Delete 1 document: 50
|
||||
ids = new ArrayList<>();
|
||||
ids.add(Integer.toString(50));
|
||||
deleteById(SOURCE_COLLECTION, ids);
|
||||
|
||||
// Index 10 documents
|
||||
docs = new ArrayList<>();
|
||||
for (; start < 70; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// Start CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
// If the non-leader node were buffering updates, then the replication must be complete
|
||||
assertEquals(59, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(59, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that batches are correctly constructed when batch boundaries are reached.
|
||||
*/
|
||||
public void doTestBatchBoundaries() throws Exception {
|
||||
invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
log.info("Indexing documents");
|
||||
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (int i = 0; i < 128; i++) { // should create two full batches (default batch = 64)
|
||||
docs.add(getDoc(id, Integer.toString(i)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
assertEquals(128, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
assertEquals(128, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(128, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check resilience of replication with delete by query executed on targets
|
||||
*/
|
||||
public void doTestResilienceWithDeleteByQueryOnTarget() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
this.clearTargetCollection();
|
||||
|
||||
// Index 50 documents
|
||||
int start = 0;
|
||||
List<SolrInputDocument> docs = new ArrayList<>();
|
||||
for (; start < 50; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// Start CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
// If the non-leader node were buffering updates, then the replication must be complete
|
||||
assertEquals(50, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(50, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
deleteByQuery(SOURCE_COLLECTION, "*:*");
|
||||
deleteByQuery(TARGET_COLLECTION, "*:*");
|
||||
|
||||
assertEquals(0, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(0, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
docs.clear();
|
||||
for (; start < 100; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
assertEquals(50, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(50, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
deleteByQuery(TARGET_COLLECTION, "*:*");
|
||||
|
||||
assertEquals(50, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(0, getNumDocs(TARGET_COLLECTION));
|
||||
|
||||
// Restart CDCR
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.STOP);
|
||||
Thread.sleep(500); // wait a bit for the state to synch
|
||||
this.invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
|
||||
docs.clear();
|
||||
for (; start < 150; start++) {
|
||||
docs.add(getDoc(id, Integer.toString(start)));
|
||||
}
|
||||
index(SOURCE_COLLECTION, docs);
|
||||
|
||||
// wait a bit for the replication to complete
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD1);
|
||||
this.waitForReplicationToComplete(SOURCE_COLLECTION, SHARD2);
|
||||
|
||||
commit(TARGET_COLLECTION);
|
||||
|
||||
assertEquals(100, getNumDocs(SOURCE_COLLECTION));
|
||||
assertEquals(50, getNumDocs(TARGET_COLLECTION));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,245 @@
|
|||
/**
|
||||
* Copyright (c) 2015 Renaud Delbru. All Rights Reserved.
|
||||
*/
|
||||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@Slow
|
||||
public class CdcrReplicationHandlerTest 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
|
||||
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
|
||||
@ShardsFixed(num = 4)
|
||||
public void doTest() throws Exception {
|
||||
this.doTestFullReplication();
|
||||
this.doTestPartialReplication();
|
||||
this.doTestPartialReplicationWithTruncatedTlog();
|
||||
this.doTestPartialReplicationAfterPeerSync();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the scenario where the slave is killed from the start. The replication
|
||||
* strategy should fetch all the missing tlog files from the leader.
|
||||
*/
|
||||
public void doTestFullReplication() throws Exception {
|
||||
List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
|
||||
ChaosMonkey.stop(slaves.get(0).jetty);
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
assertEquals(100, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
// Restart the slave node to trigger Replication strategy
|
||||
this.restartServer(slaves.get(0));
|
||||
|
||||
this.assertUpdateLogs(SOURCE_COLLECTION, 10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the scenario where the slave is killed before receiving all the documents. The replication
|
||||
* strategy should fetch all the missing tlog files from the leader.
|
||||
*/
|
||||
public void doTestPartialReplication() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
|
||||
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> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
|
||||
ChaosMonkey.stop(slaves.get(0).jetty);
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
assertEquals(200, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
// Restart the slave node to trigger Replication strategy
|
||||
this.restartServer(slaves.get(0));
|
||||
|
||||
// at this stage, the slave should have replicated the 5 missing tlog files
|
||||
this.assertUpdateLogs(SOURCE_COLLECTION, 10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the scenario where the slave is killed before receiving a commit. This creates a truncated tlog
|
||||
* file on the slave node. The replication strategy should detect this truncated file, and fetch the
|
||||
* non-truncated file from the leader.
|
||||
*/
|
||||
public void doTestPartialReplicationWithTruncatedTlog() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
|
||||
CloudSolrClient client = createCloudClient(SOURCE_COLLECTION);
|
||||
List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(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 slave in the middle of a batch to create a truncated tlog on the slave
|
||||
if (j == 45) {
|
||||
ChaosMonkey.stop(slaves.get(0).jetty);
|
||||
}
|
||||
|
||||
}
|
||||
commit(SOURCE_COLLECTION);
|
||||
}
|
||||
} finally {
|
||||
client.close();
|
||||
}
|
||||
|
||||
assertEquals(200, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
// Restart the slave node to trigger Replication recovery
|
||||
this.restartServer(slaves.get(0));
|
||||
|
||||
// at this stage, the slave should have replicated the 5 missing tlog files
|
||||
this.assertUpdateLogs(SOURCE_COLLECTION, 10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the scenario where the slave 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 slave 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.
|
||||
*/
|
||||
public void doTestPartialReplicationAfterPeerSync() throws Exception {
|
||||
this.clearSourceCollection();
|
||||
|
||||
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> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
|
||||
ChaosMonkey.stop(slaves.get(0).jetty);
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
assertEquals(100, getNumDocs(SOURCE_COLLECTION));
|
||||
|
||||
// Restart the slave node to trigger PeerSync recovery
|
||||
// (the update windows between leader and slave is small enough)
|
||||
this.restartServer(slaves.get(0));
|
||||
|
||||
ChaosMonkey.stop(slaves.get(0).jetty);
|
||||
|
||||
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 slave node to trigger Replication recovery
|
||||
this.restartServer(slaves.get(0));
|
||||
|
||||
// at this stage, the slave should have replicated the 5 missing tlog files
|
||||
this.assertUpdateLogs(SOURCE_COLLECTION, 15);
|
||||
}
|
||||
|
||||
private List<CloudJettyRunner> getShardToSlaveJetty(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 transaction logs between the leader and slave
|
||||
*/
|
||||
@Override
|
||||
protected void assertUpdateLogs(String collection, int maxNumberOfTLogs) 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> slaveFilesMeta = this.getFilesMeta(info.getReplicas(shard).get(0).ulogDir);
|
||||
|
||||
assertEquals("Incorrect number of tlog files on the leader", maxNumberOfTLogs, leaderFilesMeta.size());
|
||||
assertEquals("Incorrect number of tlog files on the slave", maxNumberOfTLogs, slaveFilesMeta.size());
|
||||
|
||||
for (Long leaderFileVersion : leaderFilesMeta.keySet()) {
|
||||
assertTrue("Slave is missing a tlog for version " + leaderFileVersion, slaveFilesMeta.containsKey(leaderFileVersion));
|
||||
assertEquals("Slave's tlog file size differs for version " + leaderFileVersion, leaderFilesMeta.get(leaderFileVersion), slaveFilesMeta.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;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.handler.CdcrParams;
|
||||
import org.junit.Test;
|
||||
|
||||
@Slow
|
||||
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();
|
||||
}
|
||||
|
||||
@Test
|
||||
@ShardsFixed(num = 4)
|
||||
|
||||
public void doTest() throws Exception {
|
||||
this.doTestLifeCycleActions();
|
||||
this.doTestCheckpointActions();
|
||||
this.doTestBufferActions();
|
||||
}
|
||||
|
||||
// check that the life-cycle state is properly synchronised across nodes
|
||||
public void doTestLifeCycleActions() throws Exception {
|
||||
// check initial status
|
||||
this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
|
||||
|
||||
// send start action to first shard
|
||||
NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.START);
|
||||
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
|
||||
public void doTestCheckpointActions() throws Exception {
|
||||
// initial request on an empty index, must return -1
|
||||
NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
|
||||
assertEquals(-1l, rsp.get(CdcrParams.CHECKPOINT));
|
||||
|
||||
index(SOURCE_COLLECTION, getDoc(id, "a")); // 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 query
|
||||
deleteByQuery(SOURCE_COLLECTION, "*:*");
|
||||
|
||||
// all the checkpoints must come from the DBQ
|
||||
rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.COLLECTIONCHECKPOINT);
|
||||
long checkpoint3 = (Long) rsp.get(CdcrParams.CHECKPOINT);
|
||||
assertTrue(checkpoint3 > 0); // ensure that checkpoints from deletes are in absolute form
|
||||
checkpoint3 = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
|
||||
assertTrue(checkpoint3 > 0); // ensure that checkpoints from deletes are in absolute form
|
||||
checkpoint3 = (Long) invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD2), CdcrParams.CdcrAction.SHARDCHECKPOINT).get(CdcrParams.CHECKPOINT);
|
||||
assertTrue(checkpoint3 > 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
|
||||
public void doTestBufferActions() throws Exception {
|
||||
// check initial status
|
||||
this.assertState(SOURCE_COLLECTION, CdcrParams.ProcessState.STOPPED, CdcrParams.BufferState.ENABLED);
|
||||
|
||||
// send disable buffer action to first shard
|
||||
NamedList rsp = invokeCdcrAction(shardToLeaderJetty.get(SOURCE_COLLECTION).get(SHARD1), CdcrParams.CdcrAction.DISABLEBUFFER);
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,304 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.util.StrUtils;
|
||||
import org.apache.solr.update.processor.CdcrUpdateProcessor;
|
||||
import org.apache.solr.update.processor.DistributedUpdateProcessor;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
||||
public class CdcrVersionReplicationTest extends BaseCdcrDistributedZkTest {
|
||||
|
||||
private static final String vfield = DistributedUpdateProcessor.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 doTest() 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -173,7 +173,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
}
|
||||
|
||||
|
||||
public static void reqhandlertests(RestTestHarness writeHarness, String testServerBaseUrl, CloudSolrClient cloudSolrServer) throws Exception {
|
||||
public static void reqhandlertests(RestTestHarness writeHarness, String testServerBaseUrl, CloudSolrClient cloudSolrClient) throws Exception {
|
||||
String payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/x', 'class': 'org.apache.solr.handler.DumpRequestHandler' , 'startup' : 'lazy'}\n" +
|
||||
"}";
|
||||
|
@ -182,7 +182,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config/overlay?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("overlay", "requestHandler", "/x", "startup"),
|
||||
"lazy",
|
||||
10);
|
||||
|
@ -195,7 +195,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config/overlay?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("overlay", "requestHandler", "/x", "a"),
|
||||
"b",
|
||||
10);
|
||||
|
@ -203,7 +203,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/x?wt=json&getdefaults=true&json.nl=map",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("getdefaults", "def_a"),
|
||||
"def A val",
|
||||
10);
|
||||
|
@ -217,7 +217,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
int maxTimeoutSeconds = 10;
|
||||
while (TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
|
||||
String uri = "/config/overlay?wt=json";
|
||||
Map m = testServerBaseUrl == null ? getRespMap(uri, writeHarness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl + uri, cloudSolrServer);
|
||||
Map m = testServerBaseUrl == null ? getRespMap(uri, writeHarness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl + uri, cloudSolrClient);
|
||||
if (null == ConfigOverlay.getObjectByPath(m, true, Arrays.asList("overlay", "requestHandler", "/x", "a"))) {
|
||||
success = true;
|
||||
break;
|
||||
|
@ -234,7 +234,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "queryConverter", "qc", "class"),
|
||||
"org.apache.solr.spelling.SpellingQueryConverter",
|
||||
10);
|
||||
|
@ -245,7 +245,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "queryConverter", "qc", "class"),
|
||||
"org.apache.solr.spelling.SuggestQueryConverter",
|
||||
10);
|
||||
|
@ -257,7 +257,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "queryConverter", "qc"),
|
||||
null,
|
||||
10);
|
||||
|
@ -269,7 +269,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "searchComponent", "tc", "class"),
|
||||
"org.apache.solr.handler.component.TermsComponent",
|
||||
10);
|
||||
|
@ -280,7 +280,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "searchComponent", "tc", "class"),
|
||||
"org.apache.solr.handler.component.TermVectorComponent",
|
||||
10);
|
||||
|
@ -292,7 +292,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "searchComponent", "tc"),
|
||||
null,
|
||||
10);
|
||||
|
@ -304,7 +304,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "valueSourceParser", "cu", "class"),
|
||||
"org.apache.solr.core.CountUsageValueSourceParser",
|
||||
10);
|
||||
|
@ -318,7 +318,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "valueSourceParser", "cu", "class"),
|
||||
"org.apache.solr.search.function.NvlValueSourceParser",
|
||||
10);
|
||||
|
@ -330,7 +330,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "valueSourceParser", "cu"),
|
||||
null,
|
||||
10);
|
||||
|
@ -344,7 +344,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "transformer", "mytrans", "class"),
|
||||
"org.apache.solr.response.transform.ValueAugmenterFactory",
|
||||
10);
|
||||
|
@ -356,7 +356,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "transformer", "mytrans", "value"),
|
||||
"6",
|
||||
10);
|
||||
|
@ -369,7 +369,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
Map map = testForResponseElement(writeHarness,
|
||||
testServerBaseUrl,
|
||||
"/config?wt=json",
|
||||
cloudSolrServer,
|
||||
cloudSolrClient,
|
||||
Arrays.asList("config", "transformer", "mytrans"),
|
||||
null,
|
||||
10);
|
||||
|
@ -383,7 +383,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
public static Map testForResponseElement(RestTestHarness harness,
|
||||
String testServerBaseUrl,
|
||||
String uri,
|
||||
CloudSolrClient cloudSolrServer, List<String> jsonPath,
|
||||
CloudSolrClient cloudSolrClient, List<String> jsonPath,
|
||||
Object expected,
|
||||
long maxTimeoutSeconds) throws Exception {
|
||||
|
||||
|
@ -393,7 +393,7 @@ public class TestSolrConfigHandler extends RestTestBase {
|
|||
|
||||
while (TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
|
||||
try {
|
||||
m = testServerBaseUrl == null ? getRespMap(uri, harness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl + uri, cloudSolrServer);
|
||||
m = testServerBaseUrl == null ? getRespMap(uri, harness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl + uri, cloudSolrClient);
|
||||
} catch (Exception e) {
|
||||
Thread.sleep(100);
|
||||
continue;
|
||||
|
|
|
@ -0,0 +1,800 @@
|
|||
package org.apache.solr.update;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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.Slow;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.update.processor.DistributedUpdateProcessor;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.noggit.ObjectBuilder;
|
||||
|
||||
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
||||
|
||||
@Slow
|
||||
public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
|
||||
|
||||
// means that we've seen the leader and have version info (i.e. we are a non-leader replica)
|
||||
private static String FROM_LEADER = DistributedUpdateProcessor.DistribPhase.FROMLEADER.toString();
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
// since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
|
||||
// since Solr can think following updates were reordered.
|
||||
@Override
|
||||
public void clearIndex() {
|
||||
try {
|
||||
deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM, FROM_LEADER));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
String response = JQ(req);
|
||||
Map rsp = (Map) ObjectBuilder.fromJSON(response);
|
||||
Map doc = null;
|
||||
if (rsp.containsKey("doc")) {
|
||||
doc = (Map) rsp.get("doc");
|
||||
} else if (rsp.containsKey("docs")) {
|
||||
List lst = (List) rsp.get("docs");
|
||||
if (lst.size() > 0) {
|
||||
doc = (Map) lst.get(0);
|
||||
}
|
||||
} else if (rsp.containsKey("response")) {
|
||||
Map responseMap = (Map) rsp.get("response");
|
||||
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);
|
||||
|
||||
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);
|
||||
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();
|
||||
|
||||
DirectUpdateHandler2.commitOnClose = false;
|
||||
final Semaphore logReplay = new Semaphore(0);
|
||||
final Semaphore logReplayFinish = new Semaphore(0);
|
||||
|
||||
UpdateLog.testing_logReplayHook = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
assertTrue(logReplay.tryAcquire(timeout, TimeUnit.SECONDS));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
UpdateLog.testing_logReplayFinishHook = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
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();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
List o = (List) reader.next();
|
||||
assertNotNull(o);
|
||||
assertTrue(o.get(2) instanceof SolrInputDocument);
|
||||
assertEquals("15", ((SolrInputDocument) o.get(2)).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 {
|
||||
DirectUpdateHandler2.commitOnClose = false;
|
||||
final Semaphore logReplayFinish = new Semaphore(0);
|
||||
UpdateLog.testing_logReplayFinishHook = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
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 {
|
||||
DirectUpdateHandler2.commitOnClose = true;
|
||||
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
|
||||
|
||||
// second delete
|
||||
o = reader.next();
|
||||
assertTrue((Long) ((List) o).get(1) < 0);
|
||||
assertTrue(reader.getLastVersion() > 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -7,9 +7,9 @@ package org.apache.solr.common.cloud;
|
|||
* 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
|
||||
|
@ -56,13 +56,13 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
* All Solr ZooKeeper interactions should go through this class rather than
|
||||
* ZooKeeper. This class handles synchronous connects and reconnections.
|
||||
*
|
||||
*/
|
||||
public class SolrZkClient implements Closeable {
|
||||
|
||||
|
||||
static final String NEWL = System.getProperty("line.separator");
|
||||
|
||||
static final int DEFAULT_CLIENT_CONNECT_TIMEOUT = 30000;
|
||||
|
@ -73,7 +73,7 @@ public class SolrZkClient implements Closeable {
|
|||
private ConnectionManager connManager;
|
||||
|
||||
private volatile SolrZooKeeper keeper;
|
||||
|
||||
|
||||
private ZkCmdExecutor zkCmdExecutor;
|
||||
|
||||
private final ExecutorService zkCallbackExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("zkCallback"));
|
||||
|
@ -90,17 +90,17 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
// expert: for tests
|
||||
public SolrZkClient() {
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout) {
|
||||
this(zkServerAddress, zkClientTimeout, new DefaultConnectionStrategy(), null);
|
||||
}
|
||||
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) {
|
||||
this(zkServerAddress, zkClientTimeout, zkClientConnectTimeout, new DefaultConnectionStrategy(), null);
|
||||
}
|
||||
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, OnReconnect onReonnect) {
|
||||
this(zkServerAddress, zkClientTimeout, zkClientConnectTimeout, new DefaultConnectionStrategy(), onReonnect);
|
||||
}
|
||||
|
@ -109,31 +109,31 @@ public class SolrZkClient implements Closeable {
|
|||
ZkClientConnectionStrategy strat, final OnReconnect onReconnect) {
|
||||
this(zkServerAddress, zkClientTimeout, DEFAULT_CLIENT_CONNECT_TIMEOUT, strat, onReconnect);
|
||||
}
|
||||
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
|
||||
ZkClientConnectionStrategy strat, final OnReconnect onReconnect) {
|
||||
this(zkServerAddress, zkClientTimeout, clientConnectTimeout, strat, onReconnect, null, null);
|
||||
}
|
||||
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
|
||||
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect) {
|
||||
this(zkServerAddress, zkClientTimeout, clientConnectTimeout, strat, onReconnect, beforeReconnect, null);
|
||||
}
|
||||
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
|
||||
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
|
||||
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider) {
|
||||
this.zkClientConnectionStrategy = strat;
|
||||
this.zkServerAddress = zkServerAddress;
|
||||
|
||||
|
||||
if (strat == null) {
|
||||
strat = new DefaultConnectionStrategy();
|
||||
}
|
||||
|
||||
|
||||
if (!strat.hasZkCredentialsToAddAutomatically()) {
|
||||
ZkCredentialsProvider zkCredentialsToAddAutomatically = createZkCredentialsToAddAutomatically();
|
||||
strat.setZkCredentialsToAddAutomatically(zkCredentialsToAddAutomatically);
|
||||
}
|
||||
|
||||
|
||||
this.zkClientTimeout = zkClientTimeout;
|
||||
// we must retry at least as long as the session timeout
|
||||
zkCmdExecutor = new ZkCmdExecutor(zkClientTimeout);
|
||||
|
@ -168,7 +168,7 @@ public class SolrZkClient implements Closeable {
|
|||
}
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
connManager.waitForConnected(clientConnectTimeout);
|
||||
} catch (Exception e) {
|
||||
|
@ -191,7 +191,7 @@ public class SolrZkClient implements Closeable {
|
|||
public ConnectionManager getConnectionManager() {
|
||||
return connManager;
|
||||
}
|
||||
|
||||
|
||||
public ZkClientConnectionStrategy getZkClientConnectionStrategy() {
|
||||
return zkClientConnectionStrategy;
|
||||
}
|
||||
|
@ -227,14 +227,14 @@ public class SolrZkClient implements Closeable {
|
|||
log.info("Using default ZkACLProvider");
|
||||
return new DefaultZkACLProvider();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns true if client is connected
|
||||
*/
|
||||
public boolean isConnected() {
|
||||
return keeper != null && keeper.getState() == ZooKeeper.States.CONNECTED;
|
||||
}
|
||||
|
||||
|
||||
public void delete(final String path, final int version, boolean retryOnConnLoss)
|
||||
throws InterruptedException, KeeperException {
|
||||
if (retryOnConnLoss) {
|
||||
|
@ -250,11 +250,16 @@ public class SolrZkClient implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private Watcher wrapWatcher (final Watcher watcher) {
|
||||
if (watcher == null) return watcher;
|
||||
/**
|
||||
* Wraps the watcher so that it doesn't fire off ZK's event queue. In order to guarantee that a watch object will
|
||||
* only be triggered once for a given notification, users need to wrap their watcher using this method before
|
||||
* calling {@link #exists(String, org.apache.zookeeper.Watcher, boolean)} or
|
||||
* {@link #getData(String, org.apache.zookeeper.Watcher, org.apache.zookeeper.data.Stat, boolean)}.
|
||||
*/
|
||||
public Watcher wrapWatcher(final Watcher watcher) {
|
||||
if (watcher == null || watcher instanceof SolrZkWatcher) return watcher;
|
||||
|
||||
// wrap the watcher so that it doesn't fire off ZK's event queue
|
||||
return new Watcher() {
|
||||
return new SolrZkWatcher() {
|
||||
@Override
|
||||
public void process(final WatchedEvent event) {
|
||||
log.debug("Submitting job to respond to event " + event);
|
||||
|
@ -268,6 +273,9 @@ public class SolrZkClient implements Closeable {
|
|||
};
|
||||
}
|
||||
|
||||
private interface SolrZkWatcher extends Watcher {
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the stat of the node of the given path. Return null if no such a
|
||||
* node exists.
|
||||
|
@ -298,7 +306,7 @@ public class SolrZkClient implements Closeable {
|
|||
return keeper.exists(path, wrapWatcher(watcher));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns true if path exists
|
||||
*/
|
||||
|
@ -366,7 +374,7 @@ public class SolrZkClient implements Closeable {
|
|||
return keeper.setData(path, data, version);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns path of created node
|
||||
*/
|
||||
|
@ -389,7 +397,7 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
/**
|
||||
* Creates the path in ZooKeeper, creating each node as necessary.
|
||||
*
|
||||
*
|
||||
* e.g. If <code>path=/solr/group/node</code> and none of the nodes, solr,
|
||||
* group, node exist, each will be created.
|
||||
*/
|
||||
|
@ -397,23 +405,23 @@ public class SolrZkClient implements Closeable {
|
|||
InterruptedException {
|
||||
makePath(path, null, CreateMode.PERSISTENT, retryOnConnLoss);
|
||||
}
|
||||
|
||||
|
||||
public void makePath(String path, boolean failOnExists, boolean retryOnConnLoss) throws KeeperException,
|
||||
InterruptedException {
|
||||
makePath(path, null, CreateMode.PERSISTENT, null, failOnExists, retryOnConnLoss);
|
||||
}
|
||||
|
||||
|
||||
public void makePath(String path, File file, boolean failOnExists, boolean retryOnConnLoss)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
makePath(path, FileUtils.readFileToByteArray(file),
|
||||
CreateMode.PERSISTENT, null, failOnExists, retryOnConnLoss);
|
||||
}
|
||||
|
||||
|
||||
public void makePath(String path, File file, boolean retryOnConnLoss) throws IOException,
|
||||
KeeperException, InterruptedException {
|
||||
makePath(path, FileUtils.readFileToByteArray(file), retryOnConnLoss);
|
||||
}
|
||||
|
||||
|
||||
public void makePath(String path, CreateMode createMode, boolean retryOnConnLoss) throws KeeperException,
|
||||
InterruptedException {
|
||||
makePath(path, null, createMode, retryOnConnLoss);
|
||||
|
@ -421,7 +429,7 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
/**
|
||||
* Creates the path in ZooKeeper, creating each node as necessary.
|
||||
*
|
||||
*
|
||||
* @param data to set on the last zkNode
|
||||
*/
|
||||
public void makePath(String path, byte[] data, boolean retryOnConnLoss) throws KeeperException,
|
||||
|
@ -431,10 +439,10 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
/**
|
||||
* Creates the path in ZooKeeper, creating each node as necessary.
|
||||
*
|
||||
*
|
||||
* e.g. If <code>path=/solr/group/node</code> and none of the nodes, solr,
|
||||
* group, node exist, each will be created.
|
||||
*
|
||||
*
|
||||
* @param data to set on the last zkNode
|
||||
*/
|
||||
public void makePath(String path, byte[] data, CreateMode createMode, boolean retryOnConnLoss)
|
||||
|
@ -444,25 +452,24 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
/**
|
||||
* Creates the path in ZooKeeper, creating each node as necessary.
|
||||
*
|
||||
*
|
||||
* e.g. If <code>path=/solr/group/node</code> and none of the nodes, solr,
|
||||
* group, node exist, each will be created.
|
||||
*
|
||||
*
|
||||
* @param data to set on the last zkNode
|
||||
*/
|
||||
public void makePath(String path, byte[] data, CreateMode createMode,
|
||||
Watcher watcher, boolean retryOnConnLoss) throws KeeperException, InterruptedException {
|
||||
makePath(path, data, createMode, watcher, true, retryOnConnLoss);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Creates the path in ZooKeeper, creating each node as necessary.
|
||||
*
|
||||
*
|
||||
* e.g. If <code>path=/solr/group/node</code> and none of the nodes, solr,
|
||||
* group, node exist, each will be created.
|
||||
*
|
||||
*
|
||||
* Note: retryOnConnLoss is only respected for the final node - nodes
|
||||
* before that are always retried on connection loss.
|
||||
*/
|
||||
|
@ -472,7 +479,7 @@ public class SolrZkClient implements Closeable {
|
|||
log.info("makePath: " + path);
|
||||
}
|
||||
boolean retry = true;
|
||||
|
||||
|
||||
if (path.startsWith("/")) {
|
||||
path = path.substring(1, path.length());
|
||||
}
|
||||
|
@ -506,7 +513,7 @@ public class SolrZkClient implements Closeable {
|
|||
keeper.create(currentPath, bytes, zkACLProvider.getACLsToAdd(currentPath), mode);
|
||||
}
|
||||
} catch (NodeExistsException e) {
|
||||
|
||||
|
||||
if (!failOnExists) {
|
||||
// TODO: version ? for now, don't worry about race
|
||||
setData(currentPath, data, -1, retryOnConnLoss);
|
||||
|
@ -514,7 +521,7 @@ public class SolrZkClient implements Closeable {
|
|||
exists(currentPath, watcher, retryOnConnLoss);
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
// ignore unless it's the last node in the path
|
||||
if (i == paths.length - 1) {
|
||||
throw e;
|
||||
|
@ -548,7 +555,7 @@ public class SolrZkClient implements Closeable {
|
|||
|
||||
/**
|
||||
* Write file to ZooKeeper - default system encoding used.
|
||||
*
|
||||
*
|
||||
* @param path path to upload file to e.g. /solr/conf/solrconfig.xml
|
||||
* @param file path to file to be uploaded
|
||||
*/
|
||||
|
@ -594,7 +601,7 @@ public class SolrZkClient implements Closeable {
|
|||
// this is the cluster state in xml format - lets pretty print
|
||||
dataString = prettyPrint(dataString);
|
||||
}
|
||||
|
||||
|
||||
string.append(dent + "DATA:\n" + dent + " "
|
||||
+ dataString.replaceAll("\n", "\n" + dent + " ") + NEWL);
|
||||
} else {
|
||||
|
@ -624,7 +631,7 @@ public class SolrZkClient implements Closeable {
|
|||
printLayout("/", 0, sb);
|
||||
System.out.println(sb.toString());
|
||||
}
|
||||
|
||||
|
||||
public static String prettyPrint(String input, int indent) {
|
||||
try {
|
||||
Source xmlInput = new StreamSource(new StringReader(input));
|
||||
|
@ -640,7 +647,7 @@ public class SolrZkClient implements Closeable {
|
|||
throw new RuntimeException("Problem pretty printing XML", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static String prettyPrint(String input) {
|
||||
return prettyPrint(input, 2);
|
||||
}
|
||||
|
@ -673,11 +680,11 @@ public class SolrZkClient implements Closeable {
|
|||
// we might have been closed already
|
||||
if (isClosed) this.keeper.close();
|
||||
}
|
||||
|
||||
|
||||
public SolrZooKeeper getSolrZooKeeper() {
|
||||
return keeper;
|
||||
}
|
||||
|
||||
|
||||
private void closeKeeper(SolrZooKeeper keeper) {
|
||||
if (keeper != null) {
|
||||
try {
|
||||
|
@ -729,7 +736,7 @@ public class SolrZkClient implements Closeable {
|
|||
return;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Validates if zkHost contains a chroot. See http://zookeeper.apache.org/doc/r3.2.2/zookeeperProgrammers.html#ch_zkSessions
|
||||
*/
|
||||
|
|
Loading…
Reference in New Issue