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:
Erick Erickson 2015-05-22 18:58:29 +00:00
parent df514f0cd0
commit 375899fdbd
34 changed files with 6947 additions and 282 deletions

View File

@ -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 ==================

View File

@ -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();
}
}

View File

@ -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);
}
}
}
}

View File

@ -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);
}
}
}
}

View File

@ -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";
}
}
}

View File

@ -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);
}
}
}
}

View File

@ -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;
}
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}
}
}

View File

@ -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;
}
}
}

View File

@ -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();
}
}
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}
}
}

View File

@ -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);
}
}

View File

@ -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.
*

View File

@ -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);
}
}
}

View File

@ -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;
}
}
}

View File

@ -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() ;

View File

@ -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.

View File

@ -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
*/

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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();

View File

@ -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();

View File

@ -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>

View File

@ -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>

View File

@ -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;
}
}
}

View File

@ -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));
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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
*/