SOLR-1873: SolrCloud - added shared/central config and core/shard managment via zookeeper, built-in load balancing, and infrastructure for future SolrCloud work.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1022188 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2010-10-13 17:01:13 +00:00
parent 54ef52a621
commit b55f338757
48 changed files with 6328 additions and 371 deletions

View File

@ -291,6 +291,10 @@ New Features
* SOLR-2010: Added ability to verify that spell checking collations have
actual results in the index. (James Dyer via gsingers)
* SOLR-1873: SolrCloud - added shared/central config and core/shard managment via zookeeper,
built-in load balancing, and infrastructure for future SolrCloud work.
(yonik, Mark Miller)
Optimizations
----------------------

View File

@ -0,0 +1,2 @@
AnyObjectId[d32987769b4827a9c539009e804415b09b39e3ed] was removed in git history.
Apache SVN contains full history.

View File

@ -0,0 +1,2 @@
AnyObjectId[3b4cca851d07d2aba1630a46b283ab63f063d7c1] was removed in git history.
Apache SVN contains full history.

View File

@ -0,0 +1,144 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.solr.common.SolrException;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
// immutable
public class CloudState {
protected static Logger log = LoggerFactory.getLogger(CloudState.class);
private final Map<String,Map<String,Slice>> collectionStates;
private final Set<String> liveNodes;
public CloudState(Set<String> liveNodes, Map<String,Map<String,Slice>> collectionStates) {
this.liveNodes = liveNodes;
this.collectionStates = collectionStates;
}
public Map<String,Slice> getSlices(String collection) {
Map<String,Slice> collectionState = collectionStates.get(collection);
if(collectionState == null) {
return null;
}
return Collections.unmodifiableMap(collectionState);
}
public Set<String> getCollections() {
return Collections.unmodifiableSet(collectionStates.keySet());
}
public Map<String,Map<String,Slice>> getCollectionStates() {
return Collections.unmodifiableMap(collectionStates);
}
public Set<String> getLiveNodes() {
return Collections.unmodifiableSet(liveNodes);
}
public boolean liveNodesContain(String name) {
return liveNodes.contains(name);
}
public static CloudState buildCloudState(SolrZkClient zkClient, CloudState oldCloudState, boolean onlyLiveNodes) throws KeeperException, InterruptedException, IOException {
Map<String,Map<String,Slice>> collectionStates;
if (!onlyLiveNodes) {
List<String> collections = zkClient.getChildren(
ZkStateReader.COLLECTIONS_ZKNODE, null);
collectionStates = new HashMap<String,Map<String,Slice>>();
for (String collection : collections) {
String shardIdPaths = ZkStateReader.COLLECTIONS_ZKNODE + "/"
+ collection + ZkStateReader.SHARDS_ZKNODE;
List<String> shardIdNames;
try {
shardIdNames = zkClient.getChildren(shardIdPaths, null);
} catch (KeeperException.NoNodeException e) {
// node is not valid currently
continue;
}
Map<String,Slice> slices = new HashMap<String,Slice>();
for (String shardIdZkPath : shardIdNames) {
Map<String,ZkNodeProps> shardsMap = readShards(zkClient, shardIdPaths
+ "/" + shardIdZkPath);
Slice slice = new Slice(shardIdZkPath, shardsMap);
slices.put(shardIdZkPath, slice);
}
collectionStates.put(collection, slices);
}
} else {
collectionStates = oldCloudState.getCollectionStates();
}
CloudState cloudInfo = new CloudState(getLiveNodes(zkClient), collectionStates);
return cloudInfo;
}
/**
* @param zkClient
* @param shardsZkPath
* @return
* @throws KeeperException
* @throws InterruptedException
* @throws IOException
*/
private static Map<String,ZkNodeProps> readShards(SolrZkClient zkClient, String shardsZkPath)
throws KeeperException, InterruptedException, IOException {
Map<String,ZkNodeProps> shardNameToProps = new HashMap<String,ZkNodeProps>();
if (zkClient.exists(shardsZkPath, null) == null) {
throw new IllegalStateException("Cannot find zk shards node that should exist:"
+ shardsZkPath);
}
List<String> shardZkPaths = zkClient.getChildren(shardsZkPath, null);
for(String shardPath : shardZkPaths) {
byte[] data = zkClient.getData(shardsZkPath + "/" + shardPath, null,
null);
ZkNodeProps props = new ZkNodeProps();
props.load(data);
shardNameToProps.put(shardPath, props);
}
return Collections.unmodifiableMap(shardNameToProps);
}
private static Set<String> getLiveNodes(SolrZkClient zkClient) throws KeeperException, InterruptedException {
List<String> liveNodes = zkClient.getChildren(ZkStateReader.LIVE_NODES_ZKNODE, null);
Set<String> liveNodesSet = new HashSet<String>(liveNodes.size());
liveNodesSet.addAll(liveNodes);
return liveNodesSet;
}
}

View File

@ -0,0 +1,140 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeoutException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class ConnectionManager implements Watcher {
protected static final Logger log = LoggerFactory
.getLogger(ConnectionManager.class);
private final String name;
private CountDownLatch clientConnected;
private KeeperState state;
private boolean connected;
private ZkClientConnectionStrategy connectionStrategy;
private String zkServerAddress;
private int zkClientTimeout;
private SolrZkClient client;
private OnReconnect onReconnect;
public ConnectionManager(String name, SolrZkClient client, String zkServerAddress, int zkClientTimeout, ZkClientConnectionStrategy strat, OnReconnect onConnect) {
this.name = name;
this.client = client;
this.connectionStrategy = strat;
this.zkServerAddress = zkServerAddress;
this.zkClientTimeout = zkClientTimeout;
this.onReconnect = onConnect;
reset();
}
private synchronized void reset() {
clientConnected = new CountDownLatch(1);
state = KeeperState.Disconnected;
connected = false;
}
public synchronized void process(WatchedEvent event) {
if (log.isInfoEnabled()) {
log.info("Watcher " + this + " name:" + name + " got event " + event
+ " path:" + event.getPath() + " type:" + event.getType());
}
state = event.getState();
if (state == KeeperState.SyncConnected) {
connected = true;
clientConnected.countDown();
} else if (state == KeeperState.Expired) {
connected = false;
log.info("Attempting to reconnect to ZooKeeper...");
try {
connectionStrategy.reconnect(zkServerAddress, zkClientTimeout, this, new ZkClientConnectionStrategy.ZkUpdate() {
@Override
public void update(SolrZooKeeper keeper) throws InterruptedException, TimeoutException, IOException {
waitForConnected(SolrZkClient.DEFAULT_CLIENT_CONNECT_TIMEOUT);
client.updateKeeper(keeper);
if(onReconnect != null) {
onReconnect.command();
}
ConnectionManager.this.connected = true;
}
});
} catch (Exception e) {
log.error("", e);
}
log.info("Connected:" + connected);
} else if (state == KeeperState.Disconnected) {
// ZooKeeper client will recover when it can
// TODO: this needs to be investigated more
connected = false;
} else {
connected = false;
}
notifyAll();
}
public synchronized boolean isConnected() {
return connected;
}
public synchronized KeeperState state() {
return state;
}
public synchronized void waitForConnected(long waitForConnection)
throws InterruptedException, TimeoutException, IOException {
long expire = System.currentTimeMillis() + waitForConnection;
long left = waitForConnection;
while (!connected && left > 0) {
wait(left);
left = expire - System.currentTimeMillis();
}
if (!connected) {
throw new TimeoutException("Could not connect to ZooKeeper " + zkServerAddress + " within " + waitForConnection + " ms");
}
}
public synchronized void waitForDisconnected(long timeout)
throws InterruptedException, TimeoutException {
long expire = System.currentTimeMillis() + timeout;
long left = timeout;
while (connected && left > 0) {
wait(left);
left = expire - System.currentTimeMillis();
}
if (connected) {
throw new TimeoutException("Did not disconnect");
}
}
}

View File

@ -0,0 +1,74 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* TODO: improve backoff retry impl
*/
public class DefaultConnectionStrategy extends ZkClientConnectionStrategy {
private static Logger log = LoggerFactory.getLogger(DefaultConnectionStrategy.class);
private ScheduledExecutorService executor;
@Override
public void connect(String serverAddress, int timeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException {
updater.update(new SolrZooKeeper(serverAddress, timeout, watcher));
}
@Override
public void reconnect(final String serverAddress, final int zkClientTimeout,
final Watcher watcher, final ZkUpdate updater) throws IOException {
log.info("Starting reconnect to ZooKeeper attempts ...");
executor = Executors.newScheduledThreadPool(1);
executor.schedule(new Runnable() {
private int delay = 1000;
public void run() {
log.info("Attempting the connect...");
boolean connected = false;
try {
updater.update(new SolrZooKeeper(serverAddress, zkClientTimeout, watcher));
log.info("Reconnected to ZooKeeper");
connected = true;
} catch (Exception e) {
log.error("", e);
log.info("Reconnect to ZooKeeper failed");
}
if(connected) {
executor.shutdownNow();
} else {
if(delay < 240000) {
delay = delay * 2;
}
executor.schedule(this, delay, TimeUnit.MILLISECONDS);
}
}
}, 1000, TimeUnit.MILLISECONDS);
}
}

View File

@ -0,0 +1,22 @@
package org.apache.solr.common.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.
*/
public interface OnReconnect {
public void command();
}

View File

@ -0,0 +1,41 @@
package org.apache.solr.common.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 java.util.Collections;
import java.util.Map;
// immutable
public class Slice {
private final Map<String,ZkNodeProps> shards;
private final String name;
public Slice(String name, Map<String,ZkNodeProps> shards) {
this.shards = shards;
this.name = name;
}
public Map<String,ZkNodeProps> getShards() {
return Collections.unmodifiableMap(shards);
}
public String getName() {
return name;
}
}

View File

@ -0,0 +1,493 @@
package org.apache.solr.common.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 java.io.File;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.List;
import java.util.concurrent.TimeoutException;
import org.apache.commons.io.FileUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*
* All Solr ZooKeeper interactions should go through this class rather than
* ZooKeeper. This class handles synchronous connects and reconnections.
*
*/
public class SolrZkClient {
static final String NEWL = System.getProperty("line.separator");
static final int DEFAULT_CLIENT_CONNECT_TIMEOUT = 30000;
private static final Logger log = LoggerFactory
.getLogger(SolrZkClient.class);
private ConnectionManager connManager;
private volatile SolrZooKeeper keeper;
/**
* @param zkServerAddress
* @param zkClientTimeout
* @throws InterruptedException
* @throws TimeoutException
* @throws IOException
*/
public SolrZkClient(String zkServerAddress, int zkClientTimeout) throws InterruptedException, TimeoutException, IOException {
this(zkServerAddress, zkClientTimeout, new DefaultConnectionStrategy(), null);
}
public SolrZkClient(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, OnReconnect onReonnect) throws InterruptedException, TimeoutException, IOException {
this(zkServerAddress, zkClientTimeout, new DefaultConnectionStrategy(), onReonnect, zkClientConnectTimeout);
}
/**
* @param zkServerAddress
* @param zkClientTimeout
* @param strat
* @param onReconnect
* @param clientConnectTimeout
* @throws InterruptedException
* @throws TimeoutException
* @throws IOException
*/
public SolrZkClient(String zkServerAddress, int zkClientTimeout,
ZkClientConnectionStrategy strat, final OnReconnect onReconnect) throws InterruptedException,
TimeoutException, IOException {
this(zkServerAddress, zkClientTimeout, strat, onReconnect, DEFAULT_CLIENT_CONNECT_TIMEOUT);
}
/**
* @param zkServerAddress
* @param zkClientTimeout
* @param strat
* @param onReconnect
* @param clientConnectTimeout
* @throws InterruptedException
* @throws TimeoutException
* @throws IOException
*/
public SolrZkClient(String zkServerAddress, int zkClientTimeout,
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, int clientConnectTimeout) throws InterruptedException,
TimeoutException, IOException {
connManager = new ConnectionManager("ZooKeeperConnection Watcher:"
+ zkServerAddress, this, zkServerAddress, zkClientTimeout, strat, onReconnect);
strat.connect(zkServerAddress, zkClientTimeout, connManager,
new ZkUpdate() {
@Override
public void update(SolrZooKeeper zooKeeper) {
SolrZooKeeper oldKeeper = keeper;
keeper = zooKeeper;
if (oldKeeper != null) {
try {
oldKeeper.close();
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
}
});
connManager.waitForConnected(clientConnectTimeout);
}
/**
* @return true if client is connected
*/
public boolean isConnected() {
return keeper != null && keeper.getState() == ZooKeeper.States.CONNECTED;
}
/**
* @param path
* @param version
* @throws InterruptedException
* @throws KeeperException
*/
public void delete(final String path, int version)
throws InterruptedException, KeeperException {
keeper.delete(path, version);
}
/**
* Return the stat of the node of the given path. Return null if no such a
* node exists.
* <p>
* If the watch is non-null and the call is successful (no exception is thrown),
* a watch will be left on the node with the given path. The watch will be
* triggered by a successful operation that creates/delete the node or sets
* the data on the node.
*
* @param path the node path
* @param watcher explicit watcher
* @return the stat of the node of the given path; return null if no such a
* node exists.
* @throws KeeperException If the server signals an error
* @throws InterruptedException If the server transaction is interrupted.
* @throws IllegalArgumentException if an invalid path is specified
*/
public Stat exists(final String path, Watcher watcher)
throws KeeperException, InterruptedException {
return keeper.exists(path, watcher);
}
/**
* @param path
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public boolean exists(final String path)
throws KeeperException, InterruptedException {
return keeper.exists(path, null) != null;
}
/**
* @param path
* @param data
* @param acl
* @param createMode
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public String create(final String path, byte data[], List<ACL> acl,
CreateMode createMode) throws KeeperException, InterruptedException {
return keeper.create(path, data, acl, createMode);
}
/**
* @param path
* @param watcher
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public List<String> getChildren(final String path, Watcher watcher)
throws KeeperException, InterruptedException {
return keeper.getChildren(path, watcher);
}
/**
* @param path
* @param watcher
* @param stat
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] getData(final String path, Watcher watcher, Stat stat)
throws KeeperException, InterruptedException {
return keeper.getData(path, watcher, stat);
}
/**
* @param path
* @param data
* @param version
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public Stat setData(final String path, byte data[], int version)
throws KeeperException, InterruptedException {
return keeper.setData(path, data, version);
}
/**
*
* @param path
* @param data
* @param watcher
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public String create(String path, byte[] data, CreateMode createMode) throws KeeperException, InterruptedException {
String zkPath = keeper.create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, createMode);
return zkPath;
}
/**
* 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 path
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String path) throws KeeperException,
InterruptedException {
makePath(path, null, CreateMode.PERSISTENT);
}
public void makePath(String path, CreateMode createMode) throws KeeperException,
InterruptedException {
makePath(path, null, createMode);
}
/**
* Creates the path in ZooKeeper, creating each node as necessary.
*
* @param path
* @param data to set on the last zkNode
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String path, byte[] data) throws KeeperException,
InterruptedException {
makePath(path, data, CreateMode.PERSISTENT);
}
/**
* 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 path
* @param data to set on the last zkNode
* @param createMode
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String path, byte[] data, CreateMode createMode)
throws KeeperException, InterruptedException {
makePath(path, data, createMode, null);
}
/**
* 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 path
* @param data to set on the last zkNode
* @param createMode
* @param watcher
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String path, byte[] data, CreateMode createMode,
Watcher watcher) throws KeeperException, InterruptedException {
makePath(path, data, createMode, watcher, false);
}
/**
* 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 path
* @param data to set on the last zkNode
* @param createMode
* @param watcher
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String path, byte[] data, CreateMode createMode,
Watcher watcher, boolean failOnExists) throws KeeperException, InterruptedException {
if (log.isInfoEnabled()) {
log.info("makePath: " + path);
}
if (path.startsWith("/")) {
path = path.substring(1, path.length());
}
String[] paths = path.split("/");
StringBuilder sbPath = new StringBuilder();
for (int i = 0; i < paths.length; i++) {
byte[] bytes = null;
String pathPiece = paths[i];
sbPath.append("/" + pathPiece);
String currentPath = sbPath.toString();
Object exists = exists(currentPath, watcher);
if (exists == null || ((i == paths.length -1) && failOnExists)) {
CreateMode mode = CreateMode.PERSISTENT;
if (i == paths.length - 1) {
mode = createMode;
bytes = data;
}
keeper.create(currentPath, bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, mode);
if(i == paths.length -1) {
// set new watch
exists(currentPath, watcher);
}
} else if (i == paths.length - 1) {
// TODO: version ? for now, don't worry about race
setData(currentPath, data, -1);
// set new watch
exists(currentPath, watcher);
}
}
}
/**
* @param zkPath
* @param createMode
* @param watcher
* @throws KeeperException
* @throws InterruptedException
*/
public void makePath(String zkPath, CreateMode createMode, Watcher watcher)
throws KeeperException, InterruptedException {
makePath(zkPath, null, createMode, watcher);
}
/**
* Write data to ZooKeeper.
*
* @param path
* @param data
* @throws KeeperException
* @throws InterruptedException
*/
public void setData(String path, byte[] data) throws KeeperException,
InterruptedException {
makePath(path);
Object exists = exists(path, null);
if (exists != null) {
setData(path, data, -1);
} else {
create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
}
}
/**
* 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
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/
public void setData(String path, File file) throws IOException,
KeeperException, InterruptedException {
if (log.isInfoEnabled()) {
log.info("Write to ZooKeepeer " + file.getAbsolutePath() + " to " + path);
}
String data = FileUtils.readFileToString(file);
setData(path, data.getBytes("UTF-8"));
}
/**
* Fills string with printout of current ZooKeeper layout.
*
* @param path
* @param indent
* @throws KeeperException
* @throws InterruptedException
*/
public void printLayout(String path, int indent, StringBuilder string)
throws KeeperException, InterruptedException {
byte[] data = getData(path, null, null);
List<String> children = getChildren(path, null);
StringBuilder dent = new StringBuilder();
for (int i = 0; i < indent; i++) {
dent.append(" ");
}
string.append(dent + path + " (" + children.size() + ")" + NEWL);
if (data != null) {
try {
String dataString = new String(data, "UTF-8");
if (!path.endsWith(".txt") && !path.endsWith(".xml")) {
string.append(dent + "DATA:\n" + dent + " "
+ dataString.replaceAll("\n", "\n" + dent + " ") + NEWL);
} else {
string.append(dent + "DATA: ...supressed..." + NEWL);
}
} catch (UnsupportedEncodingException e) {
// can't happen - UTF-8
throw new RuntimeException(e);
}
}
for (String child : children) {
if (!child.equals("quota")) {
printLayout(path + (path.equals("/") ? "" : "/") + child, indent + 1,
string);
}
}
}
/**
* Prints current ZooKeeper layout to stdout.
*
* @throws KeeperException
* @throws InterruptedException
*/
public void printLayoutToStdOut() throws KeeperException,
InterruptedException {
StringBuilder sb = new StringBuilder();
printLayout("/", 0, sb);
System.out.println(sb.toString());
}
/**
* @throws InterruptedException
*/
public void close() throws InterruptedException {
keeper.close();
}
/**
* Allows package private classes to update volatile ZooKeeper.
*
* @param keeper
* @throws InterruptedException
*/
void updateKeeper(SolrZooKeeper keeper) throws InterruptedException {
SolrZooKeeper oldKeeper = this.keeper;
this.keeper = keeper;
if (oldKeeper != null) {
oldKeeper.close();
}
}
public SolrZooKeeper getSolrZooKeeper() {
return keeper;
}
}

View File

@ -0,0 +1,37 @@
package org.apache.solr.common.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 java.io.IOException;
import org.apache.zookeeper.ClientCnxn;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
public class SolrZooKeeper extends ZooKeeper {
public SolrZooKeeper(String connectString, int sessionTimeout, Watcher watcher)
throws IOException {
super(connectString, sessionTimeout, watcher);
}
public ClientCnxn getConnection() {
return cnxn;
}
}

View File

@ -0,0 +1,36 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.concurrent.TimeoutException;
import org.apache.zookeeper.Watcher;
/**
*
*/
public abstract class ZkClientConnectionStrategy {
public abstract void connect(String zkServerAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException;
public abstract void reconnect(String serverAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException;
public static abstract class ZkUpdate {
public abstract void update(SolrZooKeeper zooKeeper) throws InterruptedException, TimeoutException, IOException;
}
}

View File

@ -0,0 +1,58 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.HashMap;
import java.util.Set;
import java.util.Map.Entry;
public class ZkNodeProps extends HashMap<String,String> {
private static final long serialVersionUID = 1L;
public void load(byte[] bytes) throws IOException {
String stringRep = new String(bytes, "UTF-8");
String[] lines = stringRep.split("\n");
for (String line : lines) {
int sepIndex = line.indexOf('=');
String key = line.substring(0, sepIndex);
String value = line.substring(sepIndex + 1, line.length());
put(key, value);
}
}
public byte[] store() throws IOException {
StringBuilder sb = new StringBuilder();
Set<Entry<String,String>> entries = entrySet();
for(Entry<String,String> entry : entries) {
sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
}
return sb.toString().getBytes("UTF-8");
}
public String toString() {
StringBuilder sb = new StringBuilder();
Set<Entry<String,String>> entries = entrySet();
for(Entry<String,String> entry : entries) {
sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
}
return sb.toString();
}
}

View File

@ -0,0 +1,400 @@
package org.apache.solr.common.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 java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.solr.common.SolrException;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ZkStateReader {
private static Logger log = LoggerFactory.getLogger(ZkStateReader.class);
public static final String COLLECTIONS_ZKNODE = "/collections";
public static final String URL_PROP = "url";
public static final String NODE_NAME = "node_name";
public static final String SHARDS_ZKNODE = "/shards";
public static final String LIVE_NODES_ZKNODE = "/live_nodes";
private volatile CloudState cloudState = new CloudState(new HashSet<String>(0), new HashMap<String,Map<String,Slice>>(0));
private static final long CLOUD_UPDATE_DELAY = Long.parseLong(System.getProperty("CLOUD_UPDATE_DELAY", "5000"));
private static class ZKTF implements ThreadFactory {
private static ThreadGroup tg = new ThreadGroup("ZkStateReader");
@Override
public Thread newThread(Runnable r) {
Thread td = new Thread(tg, r);
td.setDaemon(true);
return td;
}
}
private ScheduledExecutorService updateCloudExecutor = Executors.newScheduledThreadPool(1, new ZKTF());
private boolean cloudStateUpdateScheduled;
private SolrZkClient zkClient;
private boolean closeClient = false;
public ZkStateReader(SolrZkClient zkClient) {
this.zkClient = zkClient;
}
public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) throws InterruptedException, TimeoutException, IOException {
closeClient = true;
zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
// on reconnect, reload cloud info
new OnReconnect() {
public void command() {
try {
makeCollectionsNodeWatches();
makeShardsWatches(true);
updateCloudState(false);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
});
}
// load and publish a new CollectionInfo
public void updateCloudState(boolean immediate) throws KeeperException, InterruptedException,
IOException {
updateCloudState(immediate, false);
}
// load and publish a new CollectionInfo
public void updateLiveNodes() throws KeeperException, InterruptedException,
IOException {
updateCloudState(true, true);
}
// load and publish a new CollectionInfo
private synchronized void updateCloudState(boolean immediate, final boolean onlyLiveNodes) throws KeeperException, InterruptedException,
IOException {
// TODO: - possibly: incremental update rather than reread everything
// build immutable CloudInfo
if(immediate) {
if(!onlyLiveNodes) {
log.info("Updating cloud state from ZooKeeper... ");
} else {
log.info("Updating live nodes from ZooKeeper... ");
}
CloudState cloudState;
cloudState = CloudState.buildCloudState(zkClient, this.cloudState, onlyLiveNodes);
// update volatile
this.cloudState = cloudState;
} else {
if(cloudStateUpdateScheduled) {
log.info("Cloud state update for ZooKeeper already scheduled");
return;
}
log.info("Scheduling cloud state update from ZooKeeper...");
cloudStateUpdateScheduled = true;
updateCloudExecutor.schedule(new Runnable() {
public void run() {
log.info("Updating cloud state from ZooKeeper...");
synchronized (getUpdateLock()) {
cloudStateUpdateScheduled = false;
CloudState cloudState;
try {
cloudState = CloudState.buildCloudState(zkClient,
ZkStateReader.this.cloudState, onlyLiveNodes);
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
}
// update volatile
ZkStateReader.this.cloudState = cloudState;
}
}
}, CLOUD_UPDATE_DELAY, TimeUnit.MILLISECONDS);
}
}
public void makeShardZkNodeWatches(boolean makeWatchesForReconnect) throws KeeperException, InterruptedException {
CloudState cloudState = getCloudState();
Set<String> knownCollections = cloudState.getCollections();
List<String> collections = zkClient.getChildren(COLLECTIONS_ZKNODE, null);
for(final String collection : collections) {
if(makeWatchesForReconnect || !knownCollections.contains(collection)) {
log.info("Found new collection:" + collection);
Watcher watcher = new Watcher() {
public void process(WatchedEvent event) {
log.info("Detected changed ShardId in collection:" + collection);
try {
makeShardsWatches(collection, false);
updateCloudState(false);
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
};
boolean madeWatch = true;
String shardZkNode = COLLECTIONS_ZKNODE + "/" + collection
+ SHARDS_ZKNODE;
for (int i = 0; i < 5; i++) {
try {
zkClient.getChildren(shardZkNode, watcher);
} catch (KeeperException.NoNodeException e) {
// most likely, the collections node has been created, but not the
// shards node yet -- pause and try again
madeWatch = false;
if (i == 4) {
log.error("Could not set shards zknode watch, because the zknode does not exist:" + shardZkNode);
break;
}
Thread.sleep(100);
}
if (madeWatch) {
log.info("Made shard watch:" + shardZkNode);
break;
}
}
}
}
}
public void makeShardsWatches(final String collection, boolean makeWatchesForReconnect) throws KeeperException,
InterruptedException {
if (zkClient.exists(COLLECTIONS_ZKNODE + "/" + collection + SHARDS_ZKNODE)) {
List<String> shardIds = zkClient.getChildren(COLLECTIONS_ZKNODE + "/"
+ collection + SHARDS_ZKNODE, null);
CloudState cloudState = getCloudState();
Set<String> knownShardIds;
Map<String,Slice> slices = cloudState.getSlices(collection);
if (slices != null) {
knownShardIds = slices.keySet();
} else {
knownShardIds = new HashSet<String>(0);
}
for (final String shardId : shardIds) {
if (makeWatchesForReconnect || !knownShardIds.contains(shardId)) {
zkClient.getChildren(COLLECTIONS_ZKNODE + "/" + collection
+ SHARDS_ZKNODE + "/" + shardId, new Watcher() {
public void process(WatchedEvent event) {
log.info("Detected a shard change under ShardId:" + shardId + " in collection:" + collection);
try {
updateCloudState(false);
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
});
}
}
}
}
/**
* @throws KeeperException
* @throws InterruptedException
*/
public void makeShardsWatches(boolean makeWatchesForReconnect) throws KeeperException, InterruptedException {
List<String> collections = zkClient.getChildren(COLLECTIONS_ZKNODE, null);
for (final String collection : collections) {
makeShardsWatches(collection, makeWatchesForReconnect);
}
}
/**
* @return information about the cluster from ZooKeeper
*/
public CloudState getCloudState() {
return cloudState;
}
public Object getUpdateLock() {
return this;
}
public void close() {
if (closeClient) {
try {
zkClient.close();
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
}
}
}
public void makeCollectionsNodeWatches() throws KeeperException, InterruptedException {
log.info("Start watching collections zk node for changes");
zkClient.getChildren(ZkStateReader.COLLECTIONS_ZKNODE, new Watcher(){
public void process(WatchedEvent event) {
try {
log.info("Detected a new or removed collection");
synchronized (getUpdateLock()) {
makeShardZkNodeWatches(false);
updateCloudState(false);
}
// re-watch
String path = event.getPath();
if (path != null) {
zkClient.getChildren(path, this);
}
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}});
zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE, new Watcher(){
public void process(WatchedEvent event) {
if(event.getType() != EventType.NodeDataChanged) {
return;
}
log.info("Notified of CloudState change");
try {
synchronized (getUpdateLock()) {
makeShardZkNodeWatches(false);
updateCloudState(false);
}
zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE, this);
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}});
}
}

View File

@ -0,0 +1,33 @@
package org.apache.solr.common.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.common.SolrException;
public class ZooKeeperException extends SolrException {
public ZooKeeperException(ErrorCode code, String msg, Throwable th) {
super(code, msg, th);
}
public ZooKeeperException(ErrorCode code, String msg) {
super(code, msg);
}
}

View File

@ -59,6 +59,12 @@ public interface CoreAdminParams
* The directories are specified by multiple indexDir parameters. */
public final static String INDEX_DIR = "indexDir";
/** The collection name in solr cloud */
public final static String COLLECTION = "collection";
/** The shard id in solr cloud */
public final static String SHARD = "shard";
public enum CoreAdminAction {
STATUS,
LOAD,

View File

@ -0,0 +1,52 @@
package org.apache.solr.cloud;
import org.apache.solr.common.params.SolrParams;
/**
* 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.
*/
public class CloudDescriptor {
private String shardId;
private String collectionName;
private SolrParams params;
public void setShardId(String shardId) {
this.shardId = shardId;
}
public String getShardId() {
return shardId;
}
public String getCollectionName() {
return collectionName;
}
public void setCollectionName(String collectionName) {
this.collectionName = collectionName;
}
/** Optional parameters that can change how a core is created. */
public SolrParams getParams() {
return params;
}
public void setParams(SolrParams params) {
this.params = params;
}
}

View File

@ -0,0 +1,477 @@
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 java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.Map;
import java.util.Properties;
import java.util.Map.Entry;
import org.apache.solr.common.SolrException;
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
import org.apache.zookeeper.server.quorum.QuorumPeerMain;
import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
import org.slf4j.LoggerFactory;
public class SolrZkServer {
static org.slf4j.Logger log = LoggerFactory.getLogger(SolrZkServer.class);
String zkRun;
String zkHost;
String solrHome;
String solrPort;
Properties props;
SolrZkServerProps zkProps;
private Thread zkThread; // the thread running a zookeeper server, only if zkRun is set
public SolrZkServer(String zkRun, String zkHost, String solrHome, String solrPort) {
this.zkRun = zkRun;
this.zkHost = zkHost;
this.solrHome = solrHome;
this.solrPort = solrPort;
}
public String getClientString() {
if (zkHost != null) return zkHost;
if (zkProps == null) return null;
// if the string wasn't passed as zkHost, then use the standalone server we started
if (zkRun == null) return null;
return "localhost:" + zkProps.getClientPortAddress().getPort();
}
public void parseConfig() {
if (zkProps == null) {
zkProps = new SolrZkServerProps();
// set default data dir
// TODO: use something based on IP+port??? support ensemble all from same solr home?
zkProps.setDataDir(solrHome + '/' + "zoo_data");
zkProps.zkRun = zkRun;
zkProps.solrPort = solrPort;
}
try {
props = SolrZkServerProps.getProperties(solrHome + '/' + "zoo.cfg");
SolrZkServerProps.injectServers(props, zkRun, zkHost);
zkProps.parseProperties(props);
if (zkProps.getClientPortAddress() == null) {
zkProps.setClientPort(Integer.parseInt(solrPort)+1000);
}
} catch (QuorumPeerConfig.ConfigException e) {
if (zkRun != null)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
} catch (IOException e) {
if (zkRun != null)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public Map<Long, QuorumPeer.QuorumServer> getServers() {
return zkProps.getServers();
}
public void start() {
if (zkRun == null) return;
zkThread = new Thread() {
@Override
public void run() {
try {
if (zkProps.getServers().size() > 1) {
QuorumPeerMain zkServer = new QuorumPeerMain();
zkServer.runFromConfig(zkProps);
} else {
ServerConfig sc = new ServerConfig();
sc.readFrom(zkProps);
ZooKeeperServerMain zkServer = new ZooKeeperServerMain();
zkServer.runFromConfig(sc);
}
log.info("ZooKeeper Server exited.");
} catch (Throwable e) {
log.error("ZooKeeper Server ERROR", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
};
if (zkProps.getServers().size() > 1) {
log.info("STARTING EMBEDDED ENSEMBLE ZOOKEEPER SERVER at port " + zkProps.getClientPortAddress().getPort());
} else {
log.info("STARTING EMBEDDED STANDALONE ZOOKEEPER SERVER at port " + zkProps.getClientPortAddress().getPort());
}
zkThread.setDaemon(true);
zkThread.start();
try {
Thread.sleep(500); // pause for ZooKeeper to start
} catch (Exception e) {
log.error("STARTING ZOOKEEPER", e);
}
}
public void stop() {
if (zkRun == null) return;
zkThread.interrupt();
}
}
// Allows us to set a default for the data dir before parsing
// zoo.cfg (which validates that there is a dataDir)
class SolrZkServerProps extends QuorumPeerConfig {
protected static org.slf4j.Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class);
String solrPort; // port that Solr is listening on
String zkRun;
/**
* Parse a ZooKeeper configuration file
* @param path the patch of the configuration file
* @throws ConfigException error processing configuration
*/
public static Properties getProperties(String path) throws ConfigException {
File configFile = new File(path);
LOG.info("Reading configuration from: " + configFile);
try {
if (!configFile.exists()) {
throw new IllegalArgumentException(configFile.toString()
+ " file is missing");
}
Properties cfg = new Properties();
FileInputStream in = new FileInputStream(configFile);
try {
cfg.load(in);
} finally {
in.close();
}
return cfg;
} catch (IOException e) {
throw new ConfigException("Error processing " + path, e);
} catch (IllegalArgumentException e) {
throw new ConfigException("Error processing " + path, e);
}
}
// Adds server.x if they don't exist, based on zkHost if it does exist.
// Given zkHost=localhost:1111,localhost:2222 this will inject
// server.0=localhost:1112:1113
// server.1=localhost:2223:2224
public static void injectServers(Properties props, String zkRun, String zkHost) {
// if clientPort not already set, use zkRun
if (zkRun != null && props.getProperty("clientPort")==null) {
int portIdx = zkRun.lastIndexOf(':');
if (portIdx > 0) {
String portStr = zkRun.substring(portIdx+1);
props.setProperty("clientPort", portStr);
}
}
boolean hasServers = hasServers(props);
if (!hasServers && zkHost != null) {
int alg = Integer.parseInt(props.getProperty("electionAlg","3").trim());
String[] hosts = zkHost.split(",");
int serverNum = 0;
for (String hostAndPort : hosts) {
hostAndPort = hostAndPort.trim();
int portIdx = hostAndPort.lastIndexOf(':');
String clientPortStr = hostAndPort.substring(portIdx+1);
int clientPort = Integer.parseInt(clientPortStr);
String host = hostAndPort.substring(0,portIdx);
String serverStr = host + ':' + (clientPort+1);
// zk leader election algorithms other than 0 need an extra port for leader election.
if (alg != 0) {
serverStr = serverStr + ':' + (clientPort+2);
}
props.setProperty("server."+serverNum, serverStr);
serverNum++;
}
}
}
public static boolean hasServers(Properties props) {
for (Object key : props.keySet())
if (((String)key).startsWith("server."))
return true;
return false;
}
// called by the modified version of parseProperties
// when the myid file is missing.
public Long getMySeverId() {
if (zkRun == null && solrPort == null) return null;
Map<Long, QuorumPeer.QuorumServer> slist = getServers();
String myHost = "localhost";
InetSocketAddress thisAddr = null;
if (zkRun != null && zkRun.length()>0) {
String parts[] = zkRun.split(":");
myHost = parts[0];
thisAddr = new InetSocketAddress(myHost, Integer.parseInt(parts[1]) + 1);
} else {
// default to localhost:<solrPort+1001>
thisAddr = new InetSocketAddress(myHost, Integer.parseInt(solrPort)+1001);
}
// first try a straight match by host
Long me = null;
boolean multiple = false;
int port = 0;
for (QuorumPeer.QuorumServer server : slist.values()) {
if (server.addr.getHostName().equals(myHost)) {
multiple = me!=null;
me = server.id;
port = server.addr.getPort();
}
}
if (!multiple) {
// only one host matched... assume it's me.
setClientPort(port - 1);
return me;
}
if (me == null) {
// no hosts matched.
return null;
}
// multiple matches... try to figure out by port.
for (QuorumPeer.QuorumServer server : slist.values()) {
if (server.addr.equals(thisAddr)) {
if (clientPortAddress != null || clientPortAddress.getPort() <= 0)
setClientPort(server.addr.getPort() - 1);
return server.id;
}
}
return null;
}
public void setDataDir(String dataDir) {
this.dataDir = dataDir;
}
public void setClientPort(int clientPort) {
if (clientPortAddress != null) {
try {
this.clientPortAddress = new InetSocketAddress(
InetAddress.getByName(clientPortAddress.getHostName()), clientPort);
} catch (UnknownHostException e) {
throw new RuntimeException(e);
}
} else {
this.clientPortAddress = new InetSocketAddress(clientPort);
}
}
// NOTE: copied from ZooKeeper 3.2
/**
* Parse config from a Properties.
* @param zkProp Properties to parse from.
* @throws java.io.IOException
* @throws ConfigException
*/
public void parseProperties(Properties zkProp)
throws IOException, ConfigException {
for (Entry<Object, Object> entry : zkProp.entrySet()) {
String key = entry.getKey().toString().trim();
String value = entry.getValue().toString().trim();
if (key.equals("dataDir")) {
dataDir = value;
} else if (key.equals("dataLogDir")) {
dataLogDir = value;
} else if (key.equals("clientPort")) {
setClientPort(Integer.parseInt(value));
} else if (key.equals("tickTime")) {
tickTime = Integer.parseInt(value);
} else if (key.equals("initLimit")) {
initLimit = Integer.parseInt(value);
} else if (key.equals("syncLimit")) {
syncLimit = Integer.parseInt(value);
} else if (key.equals("electionAlg")) {
electionAlg = Integer.parseInt(value);
} else if (key.equals("maxClientCnxns")) {
maxClientCnxns = Integer.parseInt(value);
} else if (key.startsWith("server.")) {
int dot = key.indexOf('.');
long sid = Long.parseLong(key.substring(dot + 1));
String parts[] = value.split(":");
if ((parts.length != 2) && (parts.length != 3)) {
LOG.error(value
+ " does not have the form host:port or host:port:port");
}
InetSocketAddress addr = new InetSocketAddress(parts[0],
Integer.parseInt(parts[1]));
if (parts.length == 2) {
servers.put(Long.valueOf(sid), new QuorumPeer.QuorumServer(sid, addr));
} else if (parts.length == 3) {
InetSocketAddress electionAddr = new InetSocketAddress(
parts[0], Integer.parseInt(parts[2]));
servers.put(Long.valueOf(sid), new QuorumPeer.QuorumServer(sid, addr,
electionAddr));
}
} else if (key.startsWith("group")) {
int dot = key.indexOf('.');
long gid = Long.parseLong(key.substring(dot + 1));
numGroups++;
String parts[] = value.split(":");
for(String s : parts){
long sid = Long.parseLong(s);
if(serverGroup.containsKey(sid))
throw new ConfigException("Server " + sid + "is in multiple groups");
else
serverGroup.put(sid, gid);
}
} else if(key.startsWith("weight")) {
int dot = key.indexOf('.');
long sid = Long.parseLong(key.substring(dot + 1));
serverWeight.put(sid, Long.parseLong(value));
} else {
System.setProperty("zookeeper." + key, value);
}
}
if (dataDir == null) {
throw new IllegalArgumentException("dataDir is not set");
}
if (dataLogDir == null) {
dataLogDir = dataDir;
} else {
if (!new File(dataLogDir).isDirectory()) {
throw new IllegalArgumentException("dataLogDir " + dataLogDir
+ " is missing.");
}
}
if (tickTime == 0) {
throw new IllegalArgumentException("tickTime is not set");
}
if (servers.size() > 1) {
if (initLimit == 0) {
throw new IllegalArgumentException("initLimit is not set");
}
if (syncLimit == 0) {
throw new IllegalArgumentException("syncLimit is not set");
}
/*
* If using FLE, then every server requires a separate election
* port.
*/
if (electionAlg != 0) {
for (QuorumPeer.QuorumServer s : servers.values()) {
if (s.electionAddr == null)
throw new IllegalArgumentException(
"Missing election port for server: " + s.id);
}
}
/*
* Default of quorum config is majority
*/
if(serverGroup.size() > 0){
if(servers.size() != serverGroup.size())
throw new ConfigException("Every server must be in exactly one group");
/*
* The deafult weight of a server is 1
*/
for(QuorumPeer.QuorumServer s : servers.values()){
if(!serverWeight.containsKey(s.id))
serverWeight.put(s.id, (long) 1);
}
/*
* Set the quorumVerifier to be QuorumHierarchical
*/
quorumVerifier = new QuorumHierarchical(numGroups,
serverWeight, serverGroup);
} else {
/*
* The default QuorumVerifier is QuorumMaj
*/
LOG.info("Defaulting to majority quorums");
quorumVerifier = new QuorumMaj(servers.size());
}
File myIdFile = new File(dataDir, "myid");
if (!myIdFile.exists()) {
///////////////// ADDED FOR SOLR //////
Long myid = getMySeverId();
if (myid != null) {
serverId = myid;
return;
}
if (zkRun == null) return;
//////////////// END ADDED FOR SOLR //////
throw new IllegalArgumentException(myIdFile.toString()
+ " file is missing");
}
BufferedReader br = new BufferedReader(new FileReader(myIdFile));
String myIdString;
try {
myIdString = br.readLine();
} finally {
br.close();
}
try {
serverId = Long.parseLong(myIdString);
} catch (NumberFormatException e) {
throw new IllegalArgumentException("serverid " + myIdString
+ " is not a number");
}
}
}
}

View File

@ -0,0 +1,659 @@
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 java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Iterator;
import java.util.Properties;
import java.util.concurrent.TimeoutException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.SolrParams;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Handle ZooKeeper interactions.
*
* notes: loads everything on init, creates what's not there - further updates
* are prompted with Watches.
*
* TODO: exceptions during shutdown on attempts to update cloud state
*
*/
public final class ZkController {
private static Logger log = LoggerFactory.getLogger(ZkController.class);
static final String NEWL = System.getProperty("line.separator");
private final static Pattern URL_POST = Pattern.compile("https?://(.*)");
private final static Pattern URL_PREFIX = Pattern.compile("(https?://).*");
// package private for tests
static final String CONFIGS_ZKNODE = "/configs";
public final static String COLLECTION_PARAM_PREFIX="collection.";
public final static String CONFIGNAME_PROP="configName";
private SolrZkClient zkClient;
private ZkStateReader zkStateReader;
private String zkServerAddress;
private String localHostPort;
private String localHostContext;
private String localHostName;
private String localHost;
private String hostName;
/**
* @param zkServerAddress ZooKeeper server host address
* @param zkClientTimeout
* @param zkClientConnectTimeout
* @param localHost
* @param locaHostPort
* @param localHostContext
* @throws InterruptedException
* @throws TimeoutException
* @throws IOException
*/
public ZkController(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
String localHostContext) throws InterruptedException,
TimeoutException, IOException {
this.zkServerAddress = zkServerAddress;
this.localHostPort = locaHostPort;
this.localHostContext = localHostContext;
this.localHost = localHost;
zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
// on reconnect, reload cloud info
new OnReconnect() {
public void command() {
try {
zkStateReader.makeCollectionsNodeWatches();
zkStateReader.makeShardsWatches(true);
createEphemeralLiveNode();
zkStateReader.updateCloudState(false);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
});
zkStateReader = new ZkStateReader(zkClient);
init();
}
/**
* @param shardId
* @param collection
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
private void addZkShardsNode(String shardId, String collection) throws IOException, InterruptedException, KeeperException {
String shardsZkPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + ZkStateReader.SHARDS_ZKNODE + "/" + shardId;
try {
// shards node
if (!zkClient.exists(shardsZkPath)) {
if (log.isInfoEnabled()) {
log.info("creating zk shards node:" + shardsZkPath);
}
// makes shards zkNode if it doesn't exist
zkClient.makePath(shardsZkPath, CreateMode.PERSISTENT, null);
// TODO: consider how these notifications are being done
// ping that there is a new shardId
zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
}
} catch (KeeperException e) {
// its okay if another beats us creating the node
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
}
}
/**
* Closes the underlying ZooKeeper client.
*/
public void close() {
try {
zkClient.close();
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
/**
* @param collection
* @param fileName
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public boolean configFileExists(String collection, String fileName)
throws KeeperException, InterruptedException {
Stat stat = zkClient.exists(CONFIGS_ZKNODE + "/" + collection + "/" + fileName, null);
return stat != null;
}
/**
* @return information about the cluster from ZooKeeper
*/
public CloudState getCloudState() {
return zkStateReader.getCloudState();
}
/**
* @param zkConfigName
* @param fileName
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] getConfigFileData(String zkConfigName, String fileName)
throws KeeperException, InterruptedException {
String zkPath = CONFIGS_ZKNODE + "/" + zkConfigName + "/" + fileName;
byte[] bytes = zkClient.getData(zkPath, null, null);
if (bytes == null) {
log.error("Config file contains no data:" + zkPath);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"Config file contains no data:" + zkPath);
}
return bytes;
}
// TODO: consider how this is done
private String getHostAddress() throws IOException {
if (localHost == null) {
localHost = "http://" + InetAddress.getLocalHost().getHostName();
} else {
Matcher m = URL_PREFIX.matcher(localHost);
if (m.matches()) {
String prefix = m.group(1);
localHost = prefix + localHost;
} else {
localHost = "http://" + localHost;
}
}
return localHost;
}
public String getHostName() {
return hostName;
}
public SolrZkClient getZkClient() {
return zkClient;
}
/**
* @return
*/
public String getZkServerAddress() {
return zkServerAddress;
}
private void init() {
try {
localHostName = getHostAddress();
Matcher m = URL_POST.matcher(localHostName);
if (m.matches()) {
hostName = m.group(1);
} else {
log.error("Unrecognized host:" + localHostName);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"Unrecognized host:" + localHostName);
}
// makes nodes zkNode
try {
zkClient.makePath(ZkStateReader.LIVE_NODES_ZKNODE);
} catch (KeeperException e) {
// its okay if another beats us creating the node
if (e.code() != KeeperException.Code.NODEEXISTS) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
createEphemeralLiveNode();
setUpCollectionsNode();
zkStateReader.makeCollectionsNodeWatches();
} catch (IOException e) {
log.error("", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Can't create ZooKeeperController", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
private void createEphemeralLiveNode() throws KeeperException,
InterruptedException {
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
log.info("Register node as live in ZooKeeper:" + nodePath);
Watcher liveNodeWatcher = new Watcher() {
public void process(WatchedEvent event) {
try {
log.info("Updating live nodes:" + zkClient);
try {
zkStateReader.updateLiveNodes();
} finally {
// re-make watch
String path = event.getPath();
if(path == null) {
// on shutdown, it appears this can trigger with a null path
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
zkClient.getChildren(event.getPath(), this);
}
} catch (KeeperException e) {
if(e.code() == KeeperException.Code.SESSIONEXPIRED || e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
return;
}
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
};
try {
boolean nodeDeleted = true;
try {
// we attempt a delete in the case of a quick server bounce -
// if there was not a graceful shutdown, the node may exist
// until expiration timeout - so a node won't be created here because
// it exists, but eventually the node will be removed. So delete
// in case it exists and create a new node.
zkClient.delete(nodePath, -1);
} catch (KeeperException.NoNodeException e) {
// fine if there is nothing to delete
// TODO: annoying that ZK logs a warning on us
nodeDeleted = false;
}
if (nodeDeleted) {
log
.info("Found a previous node that still exists while trying to register a new live node "
+ nodePath + " - removing existing node to create another.");
}
zkClient.makePath(nodePath, CreateMode.EPHEMERAL);
} catch (KeeperException e) {
// its okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
}
zkClient.getChildren(ZkStateReader.LIVE_NODES_ZKNODE, liveNodeWatcher);
}
public String getNodeName() {
return hostName + ":" + localHostPort + "_"+ localHostContext;
}
/**
* @param path
* @return
* @throws KeeperException
* @throws InterruptedException
*/
public boolean pathExists(String path) throws KeeperException,
InterruptedException {
return zkClient.exists(path);
}
/**
* @param collection
* @return
* @throws KeeperException
* @throws InterruptedException
* @throws IOException
*/
public String readConfigName(String collection) throws KeeperException,
InterruptedException, IOException {
String configName = null;
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
if (log.isInfoEnabled()) {
log.info("Load collection config from:" + path);
}
byte[] data = zkClient.getData(path, null, null);
ZkNodeProps props = new ZkNodeProps();
if(data != null) {
props.load(data);
configName = props.get(CONFIGNAME_PROP);
}
if (configName != null && !zkClient.exists(CONFIGS_ZKNODE + "/" + configName)) {
log.error("Specified config does not exist in ZooKeeper:" + configName);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"Specified config does not exist in ZooKeeper:" + configName);
}
return configName;
}
/**
* Register shard with ZooKeeper.
*
* @param coreName
* @param cloudDesc
* @param forcePropsUpdate update solr.xml core props even if the shard is already registered
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/
public void register(String coreName, CloudDescriptor cloudDesc, boolean forcePropsUpdate) throws IOException,
KeeperException, InterruptedException {
String shardUrl = localHostName + ":" + localHostPort + "/" + localHostContext
+ "/" + coreName;
String collection = cloudDesc.getCollectionName();
String shardsZkPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + ZkStateReader.SHARDS_ZKNODE + "/" + cloudDesc.getShardId();
boolean shardZkNodeAlreadyExists = zkClient.exists(shardsZkPath);
if(shardZkNodeAlreadyExists && !forcePropsUpdate) {
return;
}
if (log.isInfoEnabled()) {
log.info("Register shard - core:" + coreName + " address:"
+ shardUrl);
}
ZkNodeProps props = new ZkNodeProps();
props.put(ZkStateReader.URL_PROP, shardUrl);
props.put(ZkStateReader.NODE_NAME, getNodeName());
byte[] bytes = props.store();
String shardZkNodeName = getNodeName() + "_" + coreName;
if(shardZkNodeAlreadyExists && forcePropsUpdate) {
zkClient.setData(shardsZkPath + "/" + shardZkNodeName, bytes);
// tell everyone to update cloud info
zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
} else {
addZkShardsNode(cloudDesc.getShardId(), collection);
try {
zkClient.create(shardsZkPath + "/" + shardZkNodeName, bytes,
CreateMode.PERSISTENT);
// tell everyone to update cloud info
zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
} catch (KeeperException e) {
// its okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
// for some reason the shard already exists, though it didn't when we
// started registration - just return
return;
}
}
}
/**
* @param coreName
* @param cloudDesc
*/
public void unregister(String coreName, CloudDescriptor cloudDesc) {
// TODO : perhaps mark the core down in zk?
}
/**
* @param dir
* @param zkPath
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/
public void uploadToZK(File dir, String zkPath) throws IOException, KeeperException, InterruptedException {
File[] files = dir.listFiles();
for(File file : files) {
if (!file.getName().startsWith(".")) {
if (!file.isDirectory()) {
zkClient.setData(zkPath + "/" + file.getName(), file);
} else {
uploadToZK(file, zkPath + "/" + file.getName());
}
}
}
}
/**
* @param dir
* @param configName
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/
public void uploadConfigDir(File dir, String configName) throws IOException, KeeperException, InterruptedException {
uploadToZK(dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
}
// convenience for testing
void printLayoutToStdOut() throws KeeperException, InterruptedException {
zkClient.printLayoutToStdOut();
}
private void setUpCollectionsNode() throws KeeperException, InterruptedException {
try {
if (!zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE)) {
if (log.isInfoEnabled()) {
log.info("creating zk collections node:" + ZkStateReader.COLLECTIONS_ZKNODE);
}
// makes collections zkNode if it doesn't exist
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE, CreateMode.PERSISTENT, null);
}
} catch (KeeperException e) {
// its okay if another beats us creating the node
if (e.code() != KeeperException.Code.NODEEXISTS) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
public void createCollectionZkNode(CloudDescriptor cd) throws KeeperException, InterruptedException, IOException {
String collection = cd.getCollectionName();
log.info("Check for collection zkNode:" + collection);
String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
try {
if(!zkClient.exists(collectionPath)) {
log.info("Creating collection in ZooKeeper:" + collection);
SolrParams params = cd.getParams();
try {
ZkNodeProps collectionProps = new ZkNodeProps();
// TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that?
String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX+CONFIGNAME_PROP, "configuration1");
// params passed in - currently only done via core admin (create core commmand).
if (params != null) {
Iterator<String> iter = params.getParameterNamesIterator();
while (iter.hasNext()) {
String paramName = iter.next();
if (paramName.startsWith(COLLECTION_PARAM_PREFIX)) {
collectionProps.put(paramName.substring(COLLECTION_PARAM_PREFIX.length()), params.get(paramName));
}
}
// if the config name wasn't passed in, use the default
if (!collectionProps.containsKey(CONFIGNAME_PROP))
collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
} else if(System.getProperty("bootstrap_confdir") != null) {
// if we are bootstrapping a collection, default the config for
// a new collection to the collection we are bootstrapping
log.info("Setting config for collection:" + collection + " to " + defaultConfigName);
Properties sysProps = System.getProperties();
for (String sprop : System.getProperties().stringPropertyNames()) {
if (sprop.startsWith(COLLECTION_PARAM_PREFIX)) {
collectionProps.put(sprop.substring(COLLECTION_PARAM_PREFIX.length()), sysProps.getProperty(sprop));
}
}
// if the config name wasn't passed in, use the default
if (!collectionProps.containsKey(CONFIGNAME_PROP))
collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
} else {
// check for configName
log.info("Looking for collection configName");
int retry = 1;
for (; retry < 6; retry++) {
if (zkClient.exists(collectionPath)) {
collectionProps = new ZkNodeProps();
collectionProps.load(zkClient.getData(collectionPath, null, null));
if (collectionProps.containsKey(CONFIGNAME_PROP)) {
break;
}
}
log.info("Could not find collection configName - pausing for 2 seconds and trying again - try: " + retry);
Thread.sleep(2000);
}
if (retry == 6) {
log.error("Could not find configName for collection " + collection);
throw new ZooKeeperException(
SolrException.ErrorCode.SERVER_ERROR,
"Could not find configName for collection " + collection);
}
}
zkClient.makePath(collectionPath, collectionProps.store(), CreateMode.PERSISTENT, null, true);
// ping that there is a new collection
zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null);
} catch (KeeperException e) {
// its okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
}
} else {
log.info("Collection zkNode exists");
}
} catch (KeeperException e) {
// its okay if another beats us creating the node
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
}
}
public ZkStateReader getZkStateReader() {
return zkStateReader;
}
}

View File

@ -0,0 +1,120 @@
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 java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.util.List;
import java.util.Properties;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.KeeperException;
/**
* ResourceLoader that works with ZooKeeper.
*
*/
public class ZkSolrResourceLoader extends SolrResourceLoader {
private final String collectionZkPath;
private ZkController zkController;
public ZkSolrResourceLoader(String instanceDir, String collection,
ZkController zooKeeperController) {
super(instanceDir);
this.zkController = zooKeeperController;
collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
}
/**
* <p>
* This loader will first attempt to load resources from ZooKeeper, but if not found
* will delegate to the context classloader when possible,
* otherwise it will attempt to resolve resources using any jar files found in
* the "lib/" directory in the specified instance directory.
* <p>
*/
public ZkSolrResourceLoader(String instanceDir, String collection, ClassLoader parent,
Properties coreProperties, ZkController zooKeeperController) {
super(instanceDir, parent, coreProperties);
this.zkController = zooKeeperController;
collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
}
/**
* Opens any resource by its name. By default, this will look in multiple
* locations to load the resource: $configDir/$resource from ZooKeeper.
* It will look for it in any jar
* accessible through the class loader if it cannot be found in ZooKeeper.
* Override this method to customize loading resources.
*
* @return the stream for the named resource
*/
public InputStream openResource(String resource) {
InputStream is = null;
String file = collectionZkPath + "/" + resource;
try {
if (zkController.pathExists(file)) {
byte[] bytes = zkController.getZkClient().getData(collectionZkPath + "/" + resource, null, null);
return new ByteArrayInputStream(bytes);
}
} catch (Exception e) {
throw new RuntimeException("Error opening " + file, e);
}
try {
// delegate to the class loader (looking into $INSTANCE_DIR/lib jars)
is = classLoader.getResourceAsStream(resource);
} catch (Exception e) {
throw new RuntimeException("Error opening " + resource, e);
}
if (is == null) {
throw new RuntimeException("Can't find resource '" + resource
+ "' in classpath or '" + collectionZkPath + "', cwd="
+ System.getProperty("user.dir"));
}
return is;
}
public String getConfigDir() {
throw new ZooKeeperException(
ErrorCode.SERVER_ERROR,
"ZkSolrResourceLoader does not support getConfigDir() - likely, what you are trying to do is not supported in ZooKeeper mode");
}
public String[] listConfigDir() {
List<String> list;
try {
list = zkController.getZkClient().getChildren(collectionZkPath, null);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
return list.toArray(new String[0]);
}
}

View File

@ -21,6 +21,7 @@ import java.io.*;
import java.nio.channels.FileChannel;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeoutException;
import java.text.SimpleDateFormat;
import org.slf4j.Logger;
@ -31,13 +32,19 @@ import javax.xml.xpath.XPathConstants;
import javax.xml.xpath.XPath;
import javax.xml.xpath.XPathExpressionException;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.SolrZkServer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.ZkSolrResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.DOMUtil;
import org.apache.solr.common.util.XML;
import org.apache.solr.common.util.FileUtils;
import org.apache.solr.handler.admin.CoreAdminHandler;
import org.apache.solr.schema.IndexSchema;
import org.apache.zookeeper.KeeperException;
import org.apache.commons.io.IOUtils;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
@ -58,6 +65,9 @@ public class CoreContainer
protected boolean persistent = false;
protected String adminPath = null;
protected String managementPath = null;
protected String hostPort;
protected String hostContext;
protected String host;
protected CoreAdminHandler coreAdminHandler = null;
protected File configFile = null;
protected String libDir = null;
@ -68,12 +78,86 @@ public class CoreContainer
protected String adminHandler;
protected boolean shareSchema;
protected String solrHome;
@Deprecated
protected String solrConfigFilenameOverride;
private String defaultCoreName = "";
private ZkController zkController;
private SolrZkServer zkServer;
private String zkHost;
public CoreContainer() {
solrHome = SolrResourceLoader.locateSolrHome();
}
private void initZooKeeper(String zkHost, int zkClientTimeout) {
// if zkHost sys property is not set, we are not using ZooKeeper
String zookeeperHost;
if(zkHost == null) {
zookeeperHost = System.getProperty("zkHost");
} else {
zookeeperHost = zkHost;
}
String zkRun = System.getProperty("zkRun");
if (zkRun == null && zookeeperHost == null)
return; // not in zk mode
zkServer = new SolrZkServer(zkRun, zookeeperHost, solrHome, hostPort);
zkServer.parseConfig();
zkServer.start();
// set client from server config if not already set
if (zookeeperHost == null) {
zookeeperHost = zkServer.getClientString();
}
int zkClientConnectTimeout = 5000;
if (zookeeperHost != null) {
// we are ZooKeeper enabled
try {
// If this is an ensemble, allow for a long connect time for other servers to come up
if (zkRun != null && zkServer.getServers().size() > 1) {
zkClientConnectTimeout = 24 * 60 * 60 * 1000; // 1 day for embedded ensemble
log.info("Zookeeper client=" + zookeeperHost + " Waiting for a quorum.");
} else {
log.info("Zookeeper client=" + zookeeperHost);
}
zkController = new ZkController(zookeeperHost, zkClientTimeout, zkClientConnectTimeout, host, hostPort, hostContext);
String confDir = System.getProperty("bootstrap_confdir");
if(confDir != null) {
File dir = new File(confDir);
if(!dir.isDirectory()) {
throw new IllegalArgumentException("bootstrap_confdir must be a directory of configuration files");
}
String confName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX+ZkController.CONFIGNAME_PROP, "configuration1");
zkController.uploadConfigDir(dir, confName);
}
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (TimeoutException e) {
log.error("Could not connect to ZooKeeper", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
}
public Properties getContainerProperties() {
return containerProperties;
@ -82,6 +166,7 @@ public class CoreContainer
// Helper class to initialize the CoreContainer
public static class Initializer {
protected String solrConfigFilename = null;
protected String dataDir = null; // override datadir for single core mode
/**
* @deprecated all cores now abort on configuration error regardless of configuration
@ -106,7 +191,8 @@ public class CoreContainer
public String getSolrConfigFilename() {
return solrConfigFilename;
}
@Deprecated
@Deprecated
public void setSolrConfigFilename(String solrConfigFilename) {
this.solrConfigFilename = solrConfigFilename;
}
@ -116,14 +202,16 @@ public class CoreContainer
ParserConfigurationException, SAXException {
CoreContainer cores = null;
String solrHome = SolrResourceLoader.locateSolrHome();
// TODO : fix broken logic confusing solr.xml with solrconfig.xml
File fconf = new File(solrHome, solrConfigFilename == null ? "solr.xml"
: solrConfigFilename);
log.info("looking for solr.xml: " + fconf.getAbsolutePath());
cores = new CoreContainer();
cores.solrConfigFilenameOverride = solrConfigFilename;
if (fconf.exists()) {
cores.load(solrHome, fconf);
} else {
log.info("no solr.xml file found - using default");
cores.load(solrHome, new ByteArrayInputStream(DEF_SOLR_XML.getBytes()));
cores.configFile = fconf;
}
@ -219,15 +307,29 @@ public class CoreContainer
if(dcoreName != null) {
defaultCoreName = dcoreName;
}
persistent = cfg.getBool( "solr/@persistent", false );
libDir = cfg.get( "solr/@sharedLib", null);
adminPath = cfg.get( "solr/cores/@adminPath", null );
shareSchema = cfg.getBool("solr/cores/@shareSchema", false );
persistent = cfg.getBool("solr/@persistent", false);
libDir = cfg.get("solr/@sharedLib", null);
zkHost = cfg.get("solr/@zkHost" , null);
adminPath = cfg.get("solr/cores/@adminPath", null);
shareSchema = cfg.getBool("solr/cores/@shareSchema", false);
int zkClientTimeout = cfg.getInt("solr/cores/@zkClientTimeout", 10000);
hostPort = System.getProperty("hostPort");
if (hostPort == null) {
hostPort = cfg.get("solr/cores/@hostPort", "8983");
}
hostContext = cfg.get("solr/cores/@hostContext", "solr");
host = cfg.get("solr/cores/@host", null);
if(shareSchema){
indexSchemaCache = new ConcurrentHashMap<String ,IndexSchema>();
}
adminHandler = cfg.get("solr/cores/@adminHandler", null );
managementPath = cfg.get("solr/cores/@managementPath", null );
zkClientTimeout = Integer.parseInt(System.getProperty("zkClientTimeout", Integer.toString(zkClientTimeout)));
initZooKeeper(zkHost, zkClientTimeout);
if (libDir != null) {
File f = FileUtils.resolvePath(new File(dir), libDir);
@ -249,25 +351,9 @@ public class CoreContainer
SolrConfig.severeErrors.add(e);
SolrException.logOnce(log,null,e);
}
// before looping over each core, let's check the names and fail
// fast if the same one is reused multiple times.
{ // local scope, won't need these vars again
NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core/@name",
XPathConstants.NODESET);
Set<String> names = new HashSet<String>();
for (int i=0; i<nodes.getLength(); i++) {
String name = DOMUtil.getText(nodes.item(i));
if (names.contains(name)) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Multiple cores found with same name: " +
name);
}
names.add(name);
}
}
NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core", XPathConstants.NODESET);
boolean defaultCoreFound = false;
for (int i=0; i<nodes.getLength(); i++) {
Node node = nodes.item(i);
try {
@ -282,12 +368,12 @@ public class CoreContainer
// be mapped to this.
name="";
}
CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, "instanceDir", null));
// deal with optional settings
String opt = DOMUtil.getAttr(node, "config", null);
if(solrConfigFilenameOverride != null && name.equals("")) {
if(solrConfigFilenameOverride != null) {
p.setConfigName(solrConfigFilenameOverride);
} else if (opt != null) {
p.setConfigName(opt);
@ -296,6 +382,16 @@ public class CoreContainer
if (opt != null) {
p.setSchemaName(opt);
}
if (zkController != null) {
opt = DOMUtil.getAttr(node, "shard", null);
if (opt != null && opt.length() > 0) {
p.getCloudDescriptor().setShardId(opt);
}
opt = DOMUtil.getAttr(node, "collection", null);
if (opt != null) {
p.getCloudDescriptor().setCollectionName(opt);
}
}
opt = DOMUtil.getAttr(node, "properties", null);
if (opt != null) {
p.setPropertiesName(opt);
@ -315,13 +411,35 @@ public class CoreContainer
SolrException.logOnce(log,null,ex);
}
}
}
finally {
} finally {
if (cfgis != null) {
try { cfgis.close(); } catch (Exception xany) {}
}
}
if(zkController != null) {
try {
synchronized (zkController.getZkStateReader().getUpdateLock()) {
zkController.getZkStateReader().makeShardZkNodeWatches(false);
zkController.getZkStateReader().updateCloudState(true);
}
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (IOException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
}
private Properties readProperties(Config cfg, Node node) throws XPathExpressionException {
@ -346,6 +464,12 @@ public class CoreContainer
}
cores.clear();
} finally {
if(zkController != null) {
zkController.close();
}
if (zkServer != null) {
zkServer.stop();
}
isShutDown = true;
}
}
@ -385,6 +509,24 @@ public class CoreContainer
core.setName(name);
}
if (zkController != null) {
try {
zkController.register(core.getName(), core.getCoreDescriptor().getCloudDescriptor(), true);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
e);
} catch (IOException e) {
log.error("", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
if( old == null || old == core) {
log.info( "registering core: "+name );
@ -427,32 +569,88 @@ public class CoreContainer
String instanceDir = idir.getPath();
// Initialize the solr config
SolrResourceLoader solrLoader = new SolrResourceLoader(instanceDir, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()));
SolrConfig config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
IndexSchema schema = null;
if(indexSchemaCache != null){
//schema sharing is enabled. so check if it already is loaded
File schemaFile = new File(dcore.getSchemaName());
if (!schemaFile.isAbsolute()) {
schemaFile = new File(solrLoader.getInstanceDir() + "conf" + File.separator + dcore.getSchemaName());
}
if(schemaFile. exists()){
String key = schemaFile.getAbsolutePath()+":"+new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(new Date(schemaFile.lastModified()));
schema = indexSchemaCache.get(key);
if(schema == null){
log.info("creating new schema object for core: " + dcore.name);
schema = new IndexSchema(config, dcore.getSchemaName(), null);
indexSchemaCache.put(key,schema);
} else {
log.info("re-using schema object for core: " + dcore.name);
SolrResourceLoader solrLoader = null;
SolrConfig config = null;
String zkConfigName = null;
if(zkController == null) {
solrLoader = new SolrResourceLoader(instanceDir, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()));
config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
} else {
try {
String collection = dcore.getCloudDescriptor().getCollectionName();
zkController.createCollectionZkNode(dcore.getCloudDescriptor());
// zkController.createCollectionZkNode(collection);
zkConfigName = zkController.readConfigName(collection);
if (zkConfigName == null) {
log.error("Could not find config name for collection:" + collection);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"Could not find config name for collection:" + collection);
}
solrLoader = new ZkSolrResourceLoader(instanceDir, zkConfigName, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()), zkController);
config = getSolrConfigFromZk(zkConfigName, dcore.getConfigName(), solrLoader);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
IndexSchema schema = null;
if (indexSchemaCache != null) {
if (zkController != null) {
File schemaFile = new File(dcore.getSchemaName());
if (!schemaFile.isAbsolute()) {
schemaFile = new File(solrLoader.getInstanceDir() + "conf"
+ File.separator + dcore.getSchemaName());
}
if (schemaFile.exists()) {
String key = schemaFile.getAbsolutePath()
+ ":"
+ new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(new Date(
schemaFile.lastModified()));
schema = indexSchemaCache.get(key);
if (schema == null) {
log.info("creating new schema object for core: " + dcore.name);
schema = new IndexSchema(config, dcore.getSchemaName(), null);
indexSchemaCache.put(key, schema);
} else {
log.info("re-using schema object for core: " + dcore.name);
}
}
} else {
// TODO: handle caching from ZooKeeper - perhaps using ZooKeepers versioning
// Don't like this cache though - how does it empty as last modified changes?
}
}
if(schema == null){
schema = new IndexSchema(config, dcore.getSchemaName(), null);
if(zkController != null) {
try {
schema = getSchemaFromZk(zkConfigName, dcore.getSchemaName(), config, solrLoader);
} catch (KeeperException e) {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
} else {
schema = new IndexSchema(config, dcore.getSchemaName(), null);
}
}
SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore);
String dataDir = null;
SolrCore core = new SolrCore(dcore.getName(), dataDir, config, schema, dcore);
return core;
}
@ -744,6 +942,18 @@ public class CoreContainer
}
opt = dcore.dataDir;
if (opt != null) writeAttribute(w,"dataDir",opt);
CloudDescriptor cd = dcore.getCloudDescriptor();
if (cd != null) {
opt = cd.getShardId();
if (opt != null)
writeAttribute(w,"shard",opt);
// only write out the collection name if it's not the default (the core name)
opt = cd.getCollectionName();
if (opt != null && !opt.equals(dcore.name))
writeAttribute(w,"collection",opt);
}
if (dcore.getCoreProperties() == null || dcore.getCoreProperties().isEmpty())
w.write("/>\n"); // core
else {
@ -801,6 +1011,37 @@ public class CoreContainer
public String getSolrHome() {
return solrHome;
}
public boolean isZooKeeperAware() {
return zkController != null;
}
public ZkController getZkController() {
return zkController;
}
private SolrConfig getSolrConfigFromZk(String zkConfigName, String solrConfigFileName,
SolrResourceLoader resourceLoader) throws IOException,
ParserConfigurationException, SAXException, KeeperException,
InterruptedException {
byte[] config = zkController.getConfigFileData(zkConfigName, solrConfigFileName);
InputStream is = new ByteArrayInputStream(config);
SolrConfig cfg = solrConfigFileName == null ? new SolrConfig(
resourceLoader, SolrConfig.DEFAULT_CONF_FILE, is) : new SolrConfig(
resourceLoader, solrConfigFileName, is);
return cfg;
}
private IndexSchema getSchemaFromZk(String zkConfigName, String schemaName,
SolrConfig config, SolrResourceLoader resourceLoader)
throws KeeperException, InterruptedException {
byte[] configBytes = zkController.getConfigFileData(zkConfigName, schemaName);
InputStream is = new ByteArrayInputStream(configBytes);
IndexSchema schema = new IndexSchema(config, schemaName, is);
return schema;
}
private static final String DEF_SOLR_XML ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
"<solr persistent=\"false\">\n" +
" <cores adminPath=\"/admin/cores\" defaultCoreName=\"" + DEFAULT_DEFAULT_CORE_NAME + "\">\n" +

View File

@ -20,6 +20,8 @@ package org.apache.solr.core;
import java.util.Properties;
import java.io.File;
import org.apache.solr.cloud.CloudDescriptor;
/**
* A Solr core descriptor
*
@ -34,10 +36,20 @@ public class CoreDescriptor {
protected String schemaName;
private final CoreContainer coreContainer;
private Properties coreProperties;
private CloudDescriptor cloudDesc;
public CoreDescriptor(CoreContainer coreContainer, String name, String instanceDir) {
this.coreContainer = coreContainer;
this.name = name;
if(coreContainer.getZkController() != null) {
this.cloudDesc = new CloudDescriptor();
// cloud collection defaults to core name
cloudDesc.setCollectionName(name == "" ? coreContainer.getDefaultCoreName() : name);
this.cloudDesc.setShardId(coreContainer.getZkController().getNodeName() + "_" + name);
}
if (name == null) {
throw new RuntimeException("Core needs a name");
}
@ -112,6 +124,10 @@ public class CoreDescriptor {
// normalize zero length to null.
if (dataDir != null && dataDir.length()==0) dataDir=null;
}
public boolean usingDefaultDataDir() {
return this.dataDir == null;
}
/**@return the core instance directory. */
public String getInstanceDir() {
@ -171,4 +187,12 @@ public class CoreDescriptor {
this.coreProperties.putAll(coreProperties);
}
}
public CloudDescriptor getCloudDescriptor() {
return cloudDesc;
}
public void setCloudDescriptor(CloudDescriptor cloudDesc) {
this.cloudDesc = cloudDesc;
}
}

View File

@ -512,7 +512,8 @@ public final class SolrCore implements SolrInfoMBean {
this.setName( name );
resourceLoader = config.getResourceLoader();
if (dataDir == null){
dataDir = config.getDataDir();
// nocommit: why did solrconfig override core descriptor !?
if(cd.usingDefaultDataDir()) dataDir = config.getDataDir();
if(dataDir == null) dataDir = cd.getDataDir();
}
@ -1556,12 +1557,10 @@ public final class SolrCore implements SolrInfoMBean {
// Hide everything...
Set<String> hide = new HashSet<String>();
File configdir = new File( solrConfig.getResourceLoader().getConfigDir() );
if( configdir.exists() && configdir.isDirectory() ) {
for( String file : configdir.list() ) {
hide.add( file.toUpperCase(Locale.ENGLISH) );
}
}
for (String file : solrConfig.getResourceLoader().listConfigDir()) {
hide.add(file.toUpperCase(Locale.ENGLISH));
}
// except the "gettable" list
StringTokenizer st = new StringTokenizer( gettable );
@ -1588,16 +1587,7 @@ public final class SolrCore implements SolrInfoMBean {
"solrconfig.xml uses deprecated <bool name='facet.sort'>. Please "+
"update your config to use <string name='facet.sort'>.");
}
if (!solrConfig.getBool("abortOnConfigurationError",true))
throw new SolrException(ErrorCode.SERVER_ERROR,
"Setting abortOnConfigurationError==false is no longer supported");
if (null != solrConfig.getVal("abortOnConfigurationError", false))
log.warn("The abortOnConfigurationError option is no longer supported "+
"in solrconfig.xml. Setting it has no effect.");
}
}
public CoreDescriptor getCoreDescriptor() {
return coreDescriptor;

View File

@ -69,7 +69,7 @@ public class SolrResourceLoader implements ResourceLoader
static final String base = "org.apache" + "." + project;
static final String[] packages = {"","analysis.","schema.","handler.","search.","update.","core.","response.","request.","update.processor.","util.", "spelling.", "handler.component.", "handler.dataimport." };
private URLClassLoader classLoader;
protected URLClassLoader classLoader;
private final String instanceDir;
private String dataDir;
@ -205,6 +205,15 @@ public class SolrResourceLoader implements ResourceLoader
public static String normalizeDir(String path) {
return ( path != null && (!(path.endsWith("/") || path.endsWith("\\"))) )? path + File.separator : path;
}
public String[] listConfigDir() {
File configdir = new File(getConfigDir());
if( configdir.exists() && configdir.isDirectory() ) {
return configdir.list();
} else {
return new String[0];
}
}
public String getConfigDir() {
return instanceDir + "conf/";

View File

@ -17,6 +17,7 @@
package org.apache.solr.handler.admin;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
@ -228,7 +229,14 @@ public class CoreAdminHandler extends RequestHandlerBase {
try {
SolrParams params = req.getParams();
String name = params.get(CoreAdminParams.NAME);
CoreDescriptor dcore = new CoreDescriptor(coreContainer, name, params.get(CoreAdminParams.INSTANCE_DIR));
String instanceDir = params.get(CoreAdminParams.INSTANCE_DIR);
if (instanceDir == null) {
// instanceDir = coreContainer.getSolrHome() + "/" + name;
instanceDir = name; // bare name is already relative to solr home
}
CoreDescriptor dcore = new CoreDescriptor(coreContainer, name, instanceDir);
// fillup optional parameters
String opts = params.get(CoreAdminParams.CONFIG);
@ -243,6 +251,19 @@ public class CoreAdminHandler extends RequestHandlerBase {
if (opts != null)
dcore.setDataDir(opts);
CloudDescriptor cd = dcore.getCloudDescriptor();
if (cd != null) {
cd.setParams(req.getParams());
opts = params.get(CoreAdminParams.COLLECTION);
if (opts != null)
cd.setCollectionName(opts);
opts = params.get(CoreAdminParams.SHARD);
if (opts != null)
cd.setShardId(opts);
}
dcore.setCoreProperties(null);
SolrCore core = coreContainer.create(dcore);
coreContainer.register(name, core, false);

View File

@ -22,12 +22,19 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.queryParser.ParseException;
import org.apache.lucene.search.*;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.*;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.FieldType;
@ -110,11 +117,116 @@ public class QueryComponent extends SearchComponent
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
// TODO: temporary... this should go in a different component.
checkDistributed(rb);
}
// TODO: this could go in a different component, or in SearchHandler
// check if this is a distributed request and set info on the response builder
void checkDistributed(ResponseBuilder rb) {
SolrQueryRequest req = rb.req;
SolrParams params = req.getParams();
rb.isDistrib = params.getBool("distrib",false);
String shards = params.get(ShardParams.SHARDS);
if (shards != null) {
List<String> lst = StrUtils.splitSmart(shards, ",", true);
rb.shards = lst.toArray(new String[lst.size()]);
// for back compat, a shards param with URLs like localhost:8983/solr will mean that this
// search is distributed.
boolean hasShardURL = shards != null && shards.indexOf('/') > 0;
rb.isDistrib = hasShardURL | rb.isDistrib;
if (rb.isDistrib) {
// since the cost of grabbing cloud state is still up in the air, we grab it only
// if we need it.
CloudState cloudState = null;
Map<String,Slice> slices = null;
CoreDescriptor coreDescriptor = req.getCore().getCoreDescriptor();
CloudDescriptor cloudDescriptor = coreDescriptor.getCloudDescriptor();
ZkController zkController = coreDescriptor.getCoreContainer().getZkController();
if (shards != null) {
List<String> lst = StrUtils.splitSmart(shards, ",", true);
rb.shards = lst.toArray(new String[lst.size()]);
rb.slices = new String[rb.shards.length];
if (zkController != null) {
// figure out which shards are slices
for (int i=0; i<rb.shards.length; i++) {
if (rb.shards[i].indexOf('/') < 0) {
// this is a logical shard
rb.slices[i] = rb.shards[i];
rb.shards[i] = null;
}
}
}
} else if (zkController != null) {
// we weren't provided with a list of slices to query, so find the list that will cover the complete index
cloudState = zkController.getCloudState();
// TODO: check "collection" for which collection(s) to search.. but for now, just default
// to the collection for this core.
// This can be more efficient... we only record the name, even though we have the
// shard info we need in the next step of mapping slice->shards
slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
/***
rb.slices = new String[slices.size()];
for (int i=0; i<rb.slices.length; i++) {
rb.slices[i] = slices.get(i).getName();
}
***/
}
//
// Map slices to shards
//
if (zkController != null) {
for (int i=0; i<rb.shards.length; i++) {
if (rb.shards[i] == null) {
if (cloudState == null) {
cloudState = zkController.getCloudState();
slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
}
String sliceName = rb.slices[i];
Slice slice = slices.get(sliceName);
if (slice==null) {
// Treat this the same as "all servers down" for a slice, and let things continue
// if partial results are acceptable
rb.shards[i] = "";
continue;
// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such shard: " + sliceName);
}
Map<String, ZkNodeProps> sliceShards = slice.getShards();
// For now, recreate the | delimited list of equivalent servers
Set<String> liveNodes = cloudState.getLiveNodes();
StringBuilder sliceShardsStr = new StringBuilder();
boolean first = true;
for (ZkNodeProps nodeProps : sliceShards.values()) {
if (!liveNodes.contains(nodeProps.get(ZkStateReader.NODE_NAME)))
continue;
if (first) {
first = false;
} else {
sliceShardsStr.append('|');
}
String url = nodeProps.get("url");
if (url.startsWith("http://"))
url = url.substring(7);
sliceShardsStr.append(url);
}
rb.shards[i] = sliceShardsStr.toString();
}
}
}
}
String shards_rows = params.get(ShardParams.SHARDS_ROWS);
if(shards_rows != null) {

View File

@ -32,7 +32,6 @@ import java.util.WeakHashMap;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.QueryElevationParams;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -48,6 +47,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.*;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.cloud.ZkController;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.SolrParams;
@ -172,19 +172,30 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
"QueryElevationComponent must specify argument: '"+CONFIG_FILE
+"' -- path to elevate.xml" );
}
File fC = new File( core.getResourceLoader().getConfigDir(), f );
File fD = new File( core.getDataDir(), f );
if( fC.exists() == fD.exists() ) {
throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
"QueryElevationComponent missing config file: '"+f + "\n"
+"either: "+fC.getAbsolutePath() + " or " + fD.getAbsolutePath() + " must exist, but not both." );
boolean exists = false;
// check if using ZooKeeper
ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
if(zkController != null) {
// TODO : shouldn't have to keep reading the config name when it has been read before
exists = zkController.configFileExists(zkController.readConfigName(core.getCoreDescriptor().getCloudDescriptor().getCollectionName()), f);
} else {
File fC = new File( core.getResourceLoader().getConfigDir(), f );
File fD = new File( core.getDataDir(), f );
if( fC.exists() == fD.exists() ) {
throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
"QueryElevationComponent missing config file: '"+f + "\n"
+"either: "+fC.getAbsolutePath() + " or " + fD.getAbsolutePath() + " must exist, but not both." );
}
if( fC.exists() ) {
exists = true;
log.info( "Loading QueryElevation from: "+ fC.getAbsolutePath() );
Config cfg = new Config( core.getResourceLoader(), f );
elevationCache.put(null, loadElevationMap( cfg ));
}
}
if( fC.exists() ) {
log.info( "Loading QueryElevation from: "+fC.getAbsolutePath() );
Config cfg = new Config( core.getResourceLoader(), f );
elevationCache.put(null, loadElevationMap( cfg ));
}
else {
if (!exists){
// preload the first data
RefCounted<SolrIndexSearcher> searchHolder = null;
try {

View File

@ -26,15 +26,11 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.DocListAndSet;
import org.apache.solr.search.QParser;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SortSpec;
import org.apache.solr.util.SolrPluginUtils;
import org.apache.solr.search.SolrIndexSearcher;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* This class is experimental and will be changing in the future.
@ -42,9 +38,8 @@ import java.util.Set;
* @version $Id$
* @since solr 1.3
*/
public class ResponseBuilder {
public class ResponseBuilder
{
public SolrQueryRequest req;
public SolrQueryResponse rsp;
public boolean doHighlights;
@ -101,7 +96,9 @@ public class ResponseBuilder {
public int stage; // What stage is this current request at?
//The address of the Shard
boolean isDistrib; // is this a distributed search?
public String[] shards;
public String[] slices; // the optional logical ids of the shards
public int shards_rows = -1;
public int shards_start = -1;
public List<ShardRequest> outgoing; // requests to be sent

View File

@ -17,32 +17,53 @@
package org.apache.solr.handler.component;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.RTimer;
import java.net.MalformedURLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
import org.apache.commons.httpclient.params.HttpMethodParams;
import org.apache.lucene.queryParser.ParseException;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
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.ShardParams;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.RTimer;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.util.SolrPluginUtils;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.apache.solr.core.SolrCore;
import org.apache.lucene.queryParser.ParseException;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
import org.apache.commons.httpclient.HttpClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
import java.util.concurrent.*;
/**
*
@ -199,7 +220,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware
subt.stop();
}
if (rb.shards == null) {
if (!rb.isDistrib) {
// a normal non-distributed request
// The semantics of debugging vs not debugging are different enough that
@ -265,6 +286,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware
for (String shard : sreq.actualShards) {
ModifiableSolrParams params = new ModifiableSolrParams(sreq.params);
params.remove(ShardParams.SHARDS); // not a top-level request
params.remove("distrib"); // not a top-level request
params.remove("indent");
params.remove(CommonParams.HEADER_ECHO_PARAMS);
params.set(ShardParams.IS_SHARD, true); // a sub (shard) request
@ -367,6 +389,8 @@ class HttpCommComponent {
static HttpClient client;
static Random r = new Random();
static LBHttpSolrServer loadbalancer;
static {
MultiThreadedHttpConnectionManager mgr = new MultiThreadedHttpConnectionManager();
@ -375,12 +399,29 @@ class HttpCommComponent {
mgr.getParams().setConnectionTimeout(SearchHandler.connectionTimeout);
mgr.getParams().setSoTimeout(SearchHandler.soTimeout);
// mgr.getParams().setStaleCheckingEnabled(false);
client = new HttpClient(mgr);
client = new HttpClient(mgr);
// prevent retries (note: this didn't work when set on mgr.. needed to be set on client)
DefaultHttpMethodRetryHandler retryhandler = new DefaultHttpMethodRetryHandler(0, false);
client.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, retryhandler);
try {
loadbalancer = new LBHttpSolrServer(client);
} catch (MalformedURLException e) {
// should be impossible since we're not passing any URLs here
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,e);
}
}
CompletionService<ShardResponse> completionService = new ExecutorCompletionService<ShardResponse>(commExecutor);
Set<Future<ShardResponse>> pending = new HashSet<Future<ShardResponse>>();
// maps "localhost:8983|localhost:7574" to a shuffled List("http://localhost:8983","http://localhost:7574")
// This is primarily to keep track of what order we should use to query the replicas of a shard
// so that we use the same replica for all phases of a distributed request.
Map<String,List<String>> shardToURLs = new HashMap<String,List<String>>();
HttpCommComponent() {
}
@ -404,7 +445,36 @@ class HttpCommComponent {
}
}
// Not thread safe... don't use in Callable.
// Don't modify the returned URL list.
private List<String> getURLs(String shard) {
List<String> urls = shardToURLs.get(shard);
if (urls==null) {
urls = StrUtils.splitSmart(shard,"|",true);
// convert shard to URL
for (int i=0; i<urls.size(); i++) {
urls.set(i, SearchHandler.scheme + urls.get(i));
}
//
// Shuffle the list instead of use round-robin by default.
// This prevents accidental synchronization where multiple shards could get in sync
// and query the same replica at the same time.
//
if (urls.size() > 1)
Collections.shuffle(urls, r);
shardToURLs.put(shard, urls);
}
return urls;
}
void submit(final ShardRequest sreq, final String shard, final ModifiableSolrParams params) {
// do this outside of the callable for thread safety reasons
final List<String> urls = getURLs(shard);
Callable<ShardResponse> task = new Callable<ShardResponse>() {
public ShardResponse call() throws Exception {
@ -416,13 +486,9 @@ class HttpCommComponent {
long startTime = System.currentTimeMillis();
try {
// String url = "http://" + shard + "/select";
String url = SearchHandler.scheme + shard;
params.remove(CommonParams.WT); // use default (currently javabin)
params.remove(CommonParams.VERSION);
SolrServer server = new CommonsHttpSolrServer(url, client);
// SolrRequest req = new QueryRequest(SolrRequest.METHOD.POST, "/select");
// use generic request to avoid extra processing of queries
QueryRequest req = new QueryRequest(params);
@ -430,10 +496,24 @@ class HttpCommComponent {
// no need to set the response parser as binary is the default
// req.setResponseParser(new BinaryResponseParser());
// srsp.rsp = server.request(req);
// srsp.rsp = server.query(sreq.params);
ssr.nl = server.request(req);
// if there are no shards available for a slice, urls.size()==0
if (urls.size()==0) {
// TODO: what's the right error code here? We should use the same thing when
// all of the servers for a shard are down.
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "no servers hosting shard: " + shard);
}
if (urls.size() <= 1) {
String url = urls.get(0);
srsp.setShardAddress(url);
SolrServer server = new CommonsHttpSolrServer(url, client);
ssr.nl = server.request(req);
} else {
LBHttpSolrServer.Rsp rsp = loadbalancer.request(new LBHttpSolrServer.Req(req, urls));
ssr.nl = rsp.getResponse();
srsp.setShardAddress(rsp.getServer());
}
} catch (Throwable th) {
srsp.setException(th);
if (th instanceof SolrException) {

View File

@ -41,8 +41,6 @@ public class ShardRequest {
public int purpose; // the purpose of this request
public String[] shards; // the shards this request should be sent to, null for all
// TODO: how to request a specific shard address?
public ModifiableSolrParams params;

View File

@ -79,4 +79,9 @@ public final class ShardResponse {
{
this.rspCode = rspCode;
}
/** What was the shard address that returned this response. Example: "http://localhost:8983/solr" */
public String getShardAddress() { return this.shardAddress; }
void setShardAddress(String addr) { this.shardAddress = addr; }
}

View File

@ -61,6 +61,7 @@ public class TermsComponent extends SearchComponent {
// TODO: temporary... this should go in a different component.
String shards = params.get(ShardParams.SHARDS);
if (shards != null) {
rb.isDistrib = true;
if (params.get(ShardParams.SHARDS_QT) == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shards.qt parameter specified");
}
@ -298,9 +299,6 @@ public class TermsComponent extends SearchComponent {
// base shard request on original parameters
sreq.params = new ModifiableSolrParams(params);
// don't pass through the shards param
sreq.params.remove(ShardParams.SHARDS);
// remove any limits for shards, we want them to return all possible
// responses
// we want this so we can calculate the correct counts
@ -310,11 +308,6 @@ public class TermsComponent extends SearchComponent {
sreq.params.set(TermsParams.TERMS_LIMIT, -1);
sreq.params.set(TermsParams.TERMS_SORT, TermsParams.TERMS_SORT_INDEX);
// TODO: is there a better way to handle this?
String qt = params.get(CommonParams.QT);
if (qt != null) {
sreq.params.add(CommonParams.QT, qt);
}
return sreq;
}

View File

@ -0,0 +1,148 @@
package org.apache.solr.client.solrj.impl;
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeoutException;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.NamedList;
import org.apache.zookeeper.KeeperException;
public class CloudSolrServer extends SolrServer {
private volatile ZkStateReader zkStateReader;
private String zkHost; // the zk server address
private int zkConnectTimeout = 10000;
private int zkClientTimeout = 10000;
private String defaultCollection;
private LBHttpSolrServer lbServer;
Random rand = new Random();
/**
* @param zkHost The address of the zookeeper quorum containing the cloud state
*/
public CloudSolrServer(String zkHost) throws MalformedURLException {
this(zkHost, new LBHttpSolrServer());
}
/**
* @param zkHost The address of the zookeeper quorum containing the cloud state
*/
public CloudSolrServer(String zkHost, LBHttpSolrServer lbServer) {
this.zkHost = zkHost;
this.lbServer = lbServer;
}
/** Sets the default collection for request */
public void setDefaultCollection(String collection) {
this.defaultCollection = collection;
}
/** Set the connect timeout to the zookeeper ensemble in ms */
public void setZkConnectTimeout(int zkConnectTimeout) {
this.zkConnectTimeout = zkConnectTimeout;
}
/** Set the timeout to the zookeeper ensemble in ms */
public void setZkClientTimeout(int zkClientTimeout) {
this.zkClientTimeout = zkClientTimeout;
}
/**
* Connect to the zookeeper ensemble.
* This is an optional method that may be used to force a connect before any other requests are sent.
*
* @throws IOException
* @throws TimeoutException
* @throws InterruptedException
*/
public void connect() {
if (zkStateReader != null) return;
synchronized(this) {
if (zkStateReader != null) return;
try {
ZkStateReader zk = new ZkStateReader(zkHost, zkConnectTimeout, zkClientTimeout);
zk.makeCollectionsNodeWatches();
zk.makeShardZkNodeWatches(false);
zk.updateCloudState(true);
zkStateReader = zk;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (KeeperException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (IOException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (TimeoutException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
}
@Override
public NamedList<Object> request(SolrRequest request) throws SolrServerException, IOException {
connect();
CloudState cloudState = zkStateReader.getCloudState();
String collection = request.getParams().get("collection", defaultCollection);
// TODO: allow multiple collections to be specified via comma separated list
Map<String,Slice> slices = cloudState.getSlices(collection);
Set<String> liveNodes = cloudState.getLiveNodes();
// IDEA: have versions on various things... like a global cloudState version
// or shardAddressVersion (which only changes when the shards change)
// to allow caching.
// build a map of unique nodes
// TODO: allow filtering by group, role, etc
Map<String,ZkNodeProps> nodes = new HashMap<String,ZkNodeProps>();
List<String> urlList = new ArrayList<String>();
for (Slice slice : slices.values()) {
for (ZkNodeProps nodeProps : slice.getShards().values()) {
String node = nodeProps.get(ZkStateReader.NODE_NAME);
if (!liveNodes.contains(node)) continue;
if (nodes.put(node, nodeProps) == null) {
String url = nodeProps.get(ZkStateReader.URL_PROP);
urlList.add(url);
}
}
}
Collections.shuffle(urlList, rand);
// System.out.println("########################## MAKING REQUEST TO " + urlList);
// TODO: set distrib=true if we detected more than one shard?
LBHttpSolrServer.Req req = new LBHttpSolrServer.Req(request, urlList);
LBHttpSolrServer.Rsp rsp = lbServer.request(req);
return rsp.getResponse();
}
public void close() {
if (zkStateReader != null) {
synchronized(this) {
if (zkStateReader!= null)
zkStateReader.close();
zkStateReader = null;
}
}
}
}

View File

@ -18,6 +18,9 @@ package org.apache.solr.client.solrj.impl;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
import org.apache.commons.httpclient.DefaultMethodRetryHandler;
import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
import org.apache.commons.httpclient.params.HttpMethodParams;
import org.apache.solr.client.solrj.*;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.util.NamedList;
@ -27,26 +30,24 @@ import java.io.IOException;
import java.lang.ref.WeakReference;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
import java.util.*;
/**
* LBHttpSolrServer or "LoadBalanced HttpSolrServer" is just a wrapper to CommonsHttpSolrServer. This is useful when you
* LBHttpSolrServer or "LoadBalanced HttpSolrServer" is a load balancing wrapper to CommonsHttpSolrServer. This is useful when you
* have multiple SolrServers and the requests need to be Load Balanced among them. This should <b>NOT</b> be used for
* indexing. Also see the <a href="http://wiki.apache.org/solr/LBHttpSolrServer">wiki</a> page.
* <p/>
* It offers automatic failover when a server goes down and it detects when the server comes back up.
* <p/>
* Load balancing is done using a simple roundrobin on the list of servers.
* Load balancing is done using a simple round-robin on the list of servers.
* <p/>
* If a request to a server fails by an IOException due to a connection timeout or read timeout then the host is taken
* off the list of live servers and moved to a 'dead server list' and the request is resent to the next live server.
* This process is continued till it tries all the live servers. If atleast one server is alive, the request succeeds,
* andif not it fails.
* and if not it fails.
* <blockquote><pre>
* SolrServer lbHttpSolrServer = new LBHttpSolrServer("http://host1:8080/solr/","http://host2:8080/solr","http://host2:8080/solr");
* //or if you wish to pass the HttpClient do as follows
@ -57,22 +58,33 @@ import java.util.concurrent.locks.ReentrantLock;
* This interval can be set using {@link #setAliveCheckInterval} , the default is set to one minute.
* <p/>
* <b>When to use this?</b><br/> This can be used as a software load balancer when you do not wish to setup an external
* load balancer. The code is relatively new and the API is currently experimental. Alternatives to this code are to use
* load balancer. Alternatives to this code are to use
* a dedicated hardware load balancer or using Apache httpd with mod_proxy_balancer as a load balancer. See <a
* href="http://en.wikipedia.org/wiki/Load_balancing_(computing)">Load balancing on Wikipedia</a>
*
* @since solr 1.4
*/
public class LBHttpSolrServer extends SolrServer {
private final CopyOnWriteArrayList<ServerWrapper> aliveServers = new CopyOnWriteArrayList<ServerWrapper>();
private final CopyOnWriteArrayList<ServerWrapper> zombieServers = new CopyOnWriteArrayList<ServerWrapper>();
// keys to the maps are currently of the form "http://localhost:8983/solr"
// which should be equivalent to CommonsHttpSolrServer.getBaseURL()
private final Map<String, ServerWrapper> aliveServers = new LinkedHashMap<String, ServerWrapper>();
// access to aliveServers should be synchronized on itself
private final Map<String, ServerWrapper> zombieServers = new ConcurrentHashMap<String, ServerWrapper>();
// changes to aliveServers are reflected in this array, no need to synchronize
private volatile ServerWrapper[] aliveServerList = new ServerWrapper[0];
private ScheduledExecutorService aliveCheckExecutor;
private HttpClient httpClient;
private final AtomicInteger counter = new AtomicInteger(-1);
private ReentrantLock checkLock = new ReentrantLock();
private static final SolrQuery solrQuery = new SolrQuery("*:*");
private static final BinaryResponseParser binaryParser = new BinaryResponseParser();
static {
solrQuery.setRows(0);
@ -81,8 +93,13 @@ public class LBHttpSolrServer extends SolrServer {
private static class ServerWrapper {
final CommonsHttpSolrServer solrServer;
// Used only by the thread in aliveCheckExecutor
long lastUsed, lastChecked;
long lastUsed; // last time used for a real request
long lastChecked; // last time checked for liveness
// "standard" servers are used by default. They normally live in the alive list
// and move to the zombie list when unavailable. When they become available again,
// they move back to the alive list.
boolean standard = true;
int failedPings = 0;
@ -93,33 +110,223 @@ public class LBHttpSolrServer extends SolrServer {
public String toString() {
return solrServer.getBaseURL();
}
public String getKey() {
return solrServer.getBaseURL();
}
@Override
public int hashCode() {
return this.getKey().hashCode();
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof ServerWrapper)) return false;
return this.getKey().equals(((ServerWrapper)obj).getKey());
}
}
public static class Req {
protected SolrRequest request;
protected List<String> servers;
protected int numDeadServersToTry;
public Req(SolrRequest request, List<String> servers) {
this.request = request;
this.servers = servers;
this.numDeadServersToTry = servers.size();
}
public SolrRequest getRequest() {
return request;
}
public List<String> getServers() {
return servers;
}
/** @return the number of dead servers to try if there are no live servers left */
public int getNumDeadServersToTry() {
return numDeadServersToTry;
}
/** @return The number of dead servers to try if there are no live servers left.
* Defaults to the number of servers in this request. */
public void setNumDeadServersToTry(int numDeadServersToTry) {
this.numDeadServersToTry = numDeadServersToTry;
}
}
public static class Rsp {
protected String server;
protected NamedList<Object> rsp;
/** The response from the server */
public NamedList<Object> getResponse() {
return rsp;
}
/** The server that returned the response */
public String getServer() {
return server;
}
}
public LBHttpSolrServer(String... solrServerUrls) throws MalformedURLException {
this(new HttpClient(new MultiThreadedHttpConnectionManager()), solrServerUrls);
DefaultHttpMethodRetryHandler retryhandler = new DefaultHttpMethodRetryHandler(0, false);
httpClient.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, retryhandler);
}
/** The provided httpClient should use a multi-threaded connection manager */
public LBHttpSolrServer(HttpClient httpClient, String... solrServerUrl)
throws MalformedURLException {
this(httpClient, new BinaryResponseParser(), solrServerUrl);
}
/** The provided httpClient should use a multi-threaded connection manager */
public LBHttpSolrServer(HttpClient httpClient, ResponseParser parser, String... solrServerUrl)
throws MalformedURLException {
this.httpClient = httpClient;
for (String s : solrServerUrl) {
aliveServers.add(new ServerWrapper(new CommonsHttpSolrServer(s, httpClient, parser)));
ServerWrapper wrapper = new ServerWrapper(makeServer(s));
aliveServers.put(wrapper.getKey(), wrapper);
}
updateAliveList();
}
public static String normalize(String server) {
if (server.endsWith("/"))
server = server.substring(0, server.length() - 1);
return server;
}
protected CommonsHttpSolrServer makeServer(String server) throws MalformedURLException {
return new CommonsHttpSolrServer(server, httpClient, binaryParser);
}
/**
* Tries to query a live server from the list provided in Req. Servers in the dead pool are skipped.
* If a request fails due to an IOException, the server is moved to the dead pool for a certain period of
* time, or until a test request on that server succeeds.
*
* Servers are queried in the exact order given (except servers currently in the dead pool are skipped).
* If no live servers from the provided list remain to be tried, a number of previously skipped dead servers will be tried.
* Req.getNumDeadServersToTry() controls how many dead servers will be tried.
*
* If no live servers are found a SolrServerException is thrown.
*
* @param req contains both the request as well as the list of servers to query
*
* @return the result of the request
*
* @throws SolrServerException
* @throws IOException
*/
public Rsp request(Req req) throws SolrServerException, IOException {
Rsp rsp = new Rsp();
Exception ex = null;
List<ServerWrapper> skipped = new ArrayList<ServerWrapper>(req.getNumDeadServersToTry());
for (String serverStr : req.getServers()) {
serverStr = normalize(serverStr);
// if the server is currently a zombie, just skip to the next one
ServerWrapper wrapper = zombieServers.get(serverStr);
if (wrapper != null) {
// System.out.println("ZOMBIE SERVER QUERIED: " + serverStr);
if (skipped.size() < req.getNumDeadServersToTry())
skipped.add(wrapper);
continue;
}
rsp.server = serverStr;
CommonsHttpSolrServer server = makeServer(serverStr);
try {
rsp.rsp = server.request(req.getRequest());
return rsp; // SUCCESS
} catch (SolrException e) {
// Server is alive but the request was malformed or invalid
throw e;
} catch (SolrServerException e) {
if (e.getRootCause() instanceof IOException) {
ex = e;
wrapper = new ServerWrapper(server);
wrapper.lastUsed = System.currentTimeMillis();
wrapper.standard = false;
zombieServers.put(wrapper.getKey(), wrapper);
startAliveCheckExecutor();
} else {
throw e;
}
} catch (Exception e) {
throw new SolrServerException(e);
}
}
// try the servers we previously skipped
for (ServerWrapper wrapper : skipped) {
try {
rsp.rsp = wrapper.solrServer.request(req.getRequest());
zombieServers.remove(wrapper.getKey());
return rsp; // SUCCESS
} catch (SolrException e) {
// Server is alive but the request was malformed or invalid
zombieServers.remove(wrapper.getKey());
throw e;
} catch (SolrServerException e) {
if (e.getRootCause() instanceof IOException) {
ex = e;
// already a zombie, no need to re-add
} else {
throw e;
}
} catch (Exception e) {
throw new SolrServerException(e);
}
}
if (ex == null) {
throw new SolrServerException("No live SolrServers available to handle this request");
} else {
throw new SolrServerException("No live SolrServers available to handle this request", ex);
}
}
private void updateAliveList() {
synchronized (aliveServers) {
aliveServerList = aliveServers.values().toArray(new ServerWrapper[aliveServers.size()]);
}
}
private ServerWrapper removeFromAlive(String key) {
synchronized (aliveServers) {
ServerWrapper wrapper = aliveServers.remove(key);
if (wrapper != null)
updateAliveList();
return wrapper;
}
}
private void addToAlive(ServerWrapper wrapper) {
synchronized (aliveServers) {
ServerWrapper prev = aliveServers.put(wrapper.getKey(), wrapper);
// TODO: warn if there was a previous entry?
updateAliveList();
}
}
public void addSolrServer(String server) throws MalformedURLException {
CommonsHttpSolrServer solrServer = new CommonsHttpSolrServer(server, httpClient);
checkLock.lock();
try {
aliveServers.add(new ServerWrapper(solrServer));
} finally {
checkLock.unlock();
}
CommonsHttpSolrServer solrServer = makeServer(server);
addToAlive(new ServerWrapper(solrServer));
}
public String removeSolrServer(String server) {
@ -131,25 +338,11 @@ public class LBHttpSolrServer extends SolrServer {
if (server.endsWith("/")) {
server = server.substring(0, server.length() - 1);
}
this.checkLock.lock();
try {
for (ServerWrapper serverWrapper : aliveServers) {
if (serverWrapper.solrServer.getBaseURL().equals(server)) {
aliveServers.remove(serverWrapper);
return serverWrapper.solrServer.getBaseURL();
}
}
if (zombieServers.isEmpty()) return null;
for (ServerWrapper serverWrapper : zombieServers) {
if (serverWrapper.solrServer.getBaseURL().equals(server)) {
zombieServers.remove(serverWrapper);
return serverWrapper.solrServer.getBaseURL();
}
}
} finally {
checkLock.unlock();
}
// there is a small race condition here - if the server is in the process of being moved between
// lists, we could fail to remove it.
removeFromAlive(server);
zombieServers.remove(server);
return null;
}
@ -173,9 +366,10 @@ public class LBHttpSolrServer extends SolrServer {
}
/**
* Tries to query a live server. If no live servers are found it throws a SolrServerException. If the request failed
* due to IOException then the live server is moved to dead pool and the request is retried on another live server if
* available. If all live servers are exhausted then a SolrServerException is thrown.
* Tries to query a live server. A SolrServerException is thrown if all servers are dead.
* If the request failed due to IOException then the live server is moved to dead pool and the request is
* retried on another live server. After live servers are exhausted, any servers previously marked as dead
* will be tried before failing the request.
*
* @param request the SolrRequest.
*
@ -186,41 +380,69 @@ public class LBHttpSolrServer extends SolrServer {
*/
public NamedList<Object> request(final SolrRequest request)
throws SolrServerException, IOException {
int count = counter.incrementAndGet();
int attempts = 0;
Exception ex;
int startSize = aliveServers.size();
while (true) {
int size = aliveServers.size();
if (size < 1) throw new SolrServerException("No live SolrServers available to handle this request");
ServerWrapper solrServer;
Exception ex = null;
ServerWrapper[] serverList = aliveServerList;
int maxTries = serverList.length;
Map<String,ServerWrapper> justFailed = null;
for (int attempts=0; attempts<maxTries; attempts++) {
int count = counter.incrementAndGet();
ServerWrapper wrapper = serverList[count % serverList.length];
wrapper.lastUsed = System.currentTimeMillis();
try {
solrServer = aliveServers.get(count % size);
} catch (IndexOutOfBoundsException e) {
//this list changes dynamically. so it is expected to get IndexOutOfBoundsException
continue;
}
try {
return solrServer.solrServer.request(request);
return wrapper.solrServer.request(request);
} catch (SolrException e) {
// Server is alive but the request was malformed or invalid
throw e;
} catch (SolrServerException e) {
if (e.getRootCause() instanceof IOException) {
ex = e;
moveAliveToDead(solrServer);
moveAliveToDead(wrapper);
if (justFailed == null) justFailed = new HashMap<String,ServerWrapper>();
justFailed.put(wrapper.getKey(), wrapper);
} else {
throw e;
}
} catch (Exception e) {
throw new SolrServerException(e);
}
attempts++;
if (attempts >= startSize)
throw new SolrServerException("No live SolrServers available to handle this request", ex);
}
// try other standard servers that we didn't try just now
for (ServerWrapper wrapper : zombieServers.values()) {
if (wrapper.standard==false || justFailed!=null && justFailed.containsKey(wrapper.getKey())) continue;
try {
NamedList<Object> rsp = wrapper.solrServer.request(request);
// remove from zombie list *before* adding to alive to avoid a race that could lose a server
zombieServers.remove(wrapper.getKey());
addToAlive(wrapper);
return rsp;
} catch (SolrException e) {
// Server is alive but the request was malformed or invalid
throw e;
} catch (SolrServerException e) {
if (e.getRootCause() instanceof IOException) {
ex = e;
// still dead
} else {
throw e;
}
} catch (Exception e) {
throw new SolrServerException(e);
}
}
if (ex == null) {
throw new SolrServerException("No live SolrServers available to handle this request");
} else {
throw new SolrServerException("No live SolrServers available to handle this request", ex);
}
}
/**
* Takes up one dead server and check for aliveness. The check is done in a roundrobin. Each server is checked for
* aliveness once in 'x' millis where x is decided by the setAliveCheckinterval() or it is defaulted to 1 minute
@ -229,37 +451,42 @@ public class LBHttpSolrServer extends SolrServer {
*/
private void checkAZombieServer(ServerWrapper zombieServer) {
long currTime = System.currentTimeMillis();
checkLock.lock();
try {
zombieServer.lastChecked = currTime;
QueryResponse resp = zombieServer.solrServer.query(solrQuery);
if (resp.getStatus() == 0) {
//server has come back up
zombieServer.lastUsed = currTime;
zombieServers.remove(zombieServer);
aliveServers.add(zombieServer);
zombieServer.failedPings = 0;
// server has come back up.
// make sure to remove from zombies before adding to alive to avoid a race condition
// where another thread could mark it down, move it back to zombie, and then we delete
// from zombie and lose it forever.
ServerWrapper wrapper = zombieServers.remove(zombieServer.getKey());
if (wrapper != null) {
wrapper.failedPings = 0;
if (wrapper.standard) {
addToAlive(wrapper);
}
} else {
// something else already moved the server from zombie to alive
}
}
} catch (Exception e) {
//Expected. The server is still down.
zombieServer.failedPings++;
//Expected . The server is still down
} finally {
checkLock.unlock();
// If the server doesn't belong in the standard set belonging to this load balancer
// then simply drop it after a certain number of failed pings.
if (!zombieServer.standard && zombieServer.failedPings >= NONSTANDARD_PING_LIMIT) {
zombieServers.remove(zombieServer.getKey());
}
}
}
private void moveAliveToDead(ServerWrapper solrServer) {
checkLock.lock();
try {
boolean result = aliveServers.remove(solrServer);
if (result) {
if (zombieServers.addIfAbsent(solrServer)) {
startAliveCheckExecutor();
}
}
} finally {
checkLock.unlock();
}
private void moveAliveToDead(ServerWrapper wrapper) {
wrapper = removeFromAlive(wrapper.getKey());
if (wrapper == null)
return; // another thread already detected the failure and removed it
zombieServers.put(wrapper.getKey(), wrapper);
startAliveCheckExecutor();
}
private int interval = CHECK_INTERVAL;
@ -279,6 +506,8 @@ public class LBHttpSolrServer extends SolrServer {
}
private void startAliveCheckExecutor() {
// double-checked locking, but it's OK because we don't *do* anything with aliveCheckExecutor
// if it's not null.
if (aliveCheckExecutor == null) {
synchronized (this) {
if (aliveCheckExecutor == null) {
@ -291,13 +520,13 @@ public class LBHttpSolrServer extends SolrServer {
}
}
private static Runnable getAliveCheckRunner(final WeakReference<LBHttpSolrServer> lbHttpSolrServer) {
private static Runnable getAliveCheckRunner(final WeakReference<LBHttpSolrServer> lbRef) {
return new Runnable() {
public void run() {
LBHttpSolrServer solrServer = lbHttpSolrServer.get();
if (solrServer != null && solrServer.zombieServers != null) {
for (ServerWrapper zombieServer : solrServer.zombieServers) {
solrServer.checkAZombieServer(zombieServer);
LBHttpSolrServer lb = lbRef.get();
if (lb != null && lb.zombieServers != null) {
for (ServerWrapper zombieServer : lb.zombieServers.values()) {
lb.checkAZombieServer(zombieServer);
}
}
}
@ -317,5 +546,7 @@ public class LBHttpSolrServer extends SolrServer {
}
}
// defaults
private static final int CHECK_INTERVAL = 60 * 1000; //1 minute between checks
private static final int NONSTANDARD_PING_LIMIT = 5; // number of times we'll ping dead servers not in the server list
}

View File

@ -1,6 +1,20 @@
package org.apache.solr;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import junit.framework.TestCase;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@ -17,10 +31,6 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.*;
/**
* Helper base class for distributed search test cases
*
@ -44,6 +54,8 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
protected List<JettySolrRunner> jettys = new ArrayList<JettySolrRunner>();
protected String context = "/solr";
protected String shards;
protected String[] shardsArr;
protected String[] deadServers = {"does_not_exist_54321.com:33331/solr","localhost:33332/solr"};
protected File testDir;
protected SolrServer controlClient;
@ -130,40 +142,79 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
super.tearDown();
}
private void createServers(int numShards) throws Exception {
controlJetty = createJetty(testDir, "control");
protected void createServers(int numShards) throws Exception {
controlJetty = createJetty(testDir, testDir + "/control/data");
controlClient = createNewSolrServer(controlJetty.getLocalPort());
shardsArr = new String[numShards];
StringBuilder sb = new StringBuilder();
for (int i = 1; i <= numShards; i++) {
for (int i = 0; i < numShards; i++) {
if (sb.length() > 0) sb.append(',');
JettySolrRunner j = createJetty(testDir, "shard" + i);
JettySolrRunner j = createJetty(testDir, testDir + "/shard" + i + "/data");
jettys.add(j);
clients.add(createNewSolrServer(j.getLocalPort()));
sb.append("localhost:").append(j.getLocalPort()).append(context);
String shardStr = "localhost:" + j.getLocalPort() + context;
shardsArr[i] = shardStr;
sb.append(shardStr);
}
shards = sb.toString();
}
protected void setDistributedParams(ModifiableSolrParams params) {
params.set("shards", getShardsString());
}
protected String getShardsString() {
if (deadServers == null) return shards;
StringBuilder sb = new StringBuilder();
for (String shard : shardsArr) {
if (sb.length() > 0) sb.append(',');
int nDeadServers = r.nextInt(deadServers.length+1);
if (nDeadServers > 0) {
List<String> replicas = new ArrayList<String>(Arrays.asList(deadServers));
Collections.shuffle(replicas, r);
replicas.add(r.nextInt(nDeadServers+1), shard);
for (int i=0; i<nDeadServers+1; i++) {
if (i!=0) sb.append('|');
sb.append(replicas.get(i));
}
} else {
sb.append(shard);
}
}
return sb.toString();
}
protected void destroyServers() throws Exception {
controlJetty.stop();
for (JettySolrRunner jetty : jettys) jetty.stop();
clients.clear();
jettys.clear();
}
public static JettySolrRunner createJetty(File baseDir, String dataDirName) throws Exception {
File subDir = new File(baseDir, dataDirName);
subDir.mkdirs();
System.setProperty("solr.data.dir", subDir.toString());
JettySolrRunner jetty = new JettySolrRunner("/solr", 0);
jetty.start();
return jetty;
public JettySolrRunner createJetty(File baseDir, String dataDir) throws Exception {
return createJetty(baseDir, dataDir, null, null);
}
public JettySolrRunner createJetty(File baseDir, String dataDir, String shardId) throws Exception {
return createJetty(baseDir, dataDir, shardId, null);
}
public JettySolrRunner createJetty(File baseDir, String dataDir, String shardList, String solrConfigOverride) throws Exception {
System.setProperty("solr.data.dir", dataDir);
JettySolrRunner jetty = new JettySolrRunner("/solr", 0, solrConfigOverride);
if(shardList != null) {
System.setProperty("shard", shardList);
}
jetty.start();
System.clearProperty("shard");
return jetty;
}
protected SolrServer createNewSolrServer(int port) {
try {
// setup the server...
@ -230,6 +281,14 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
for (SolrServer client : clients) client.commit();
}
protected QueryResponse queryServer(ModifiableSolrParams params) throws SolrServerException {
// query a random server
int which = r.nextInt(clients.size());
SolrServer client = clients.get(which);
QueryResponse rsp = client.query(params);
return rsp;
}
protected void query(Object... q) throws Exception {
final ModifiableSolrParams params = new ModifiableSolrParams();
@ -239,11 +298,9 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
final QueryResponse controlRsp = controlClient.query(params);
// query a random server
params.set("shards", shards);
int which = r.nextInt(clients.size());
SolrServer client = clients.get(which);
QueryResponse rsp = client.query(params);
setDistributedParams(params);
QueryResponse rsp = queryServer(params);
compareResponses(rsp, controlRsp);

View File

@ -0,0 +1,92 @@
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 java.io.File;
import org.apache.solr.BaseDistributedSearchTestCase;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.core.SolrConfig;
import org.junit.Before;
public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearchTestCase {
private static final boolean DEBUG = false;
protected ZkTestServer zkServer;
@Before
@Override
public void setUp() throws Exception {
super.setUp();
log.info("####SETUP_START " + getName());
ignoreException("java.nio.channels.ClosedChannelException");
String zkDir = testDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
zkServer = new ZkTestServer(zkDir);
zkServer.run();
System.setProperty("zkHost", zkServer.getZkAddress());
AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(), "solrconfig.xml", "schema.xml");
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
}
protected void createServers(int numShards) throws Exception {
System.setProperty("collection", "control_collection");
controlJetty = createJetty(testDir, testDir + "/control/data", "control_shard");
System.clearProperty("collection");
controlClient = createNewSolrServer(controlJetty.getLocalPort());
StringBuilder sb = new StringBuilder();
for (int i = 1; i <= numShards; i++) {
if (sb.length() > 0) sb.append(',');
JettySolrRunner j = createJetty(testDir, testDir + "/jetty" + i, "shard" + (i + 2));
jettys.add(j);
clients.add(createNewSolrServer(j.getLocalPort()));
sb.append("localhost:").append(j.getLocalPort()).append(context);
}
shards = sb.toString();
}
@Override
public void tearDown() throws Exception {
if (DEBUG) {
printLayout();
}
zkServer.shutdown();
System.clearProperty("zkHost");
System.clearProperty("collection");
System.clearProperty("solr.test.sys.prop1");
System.clearProperty("solr.test.sys.prop2");
super.tearDown();
resetExceptionIgnores();
SolrConfig.severeErrors.clear();
}
protected void printLayout() throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT);
zkClient.printLayoutToStdOut();
zkClient.close();
}
}

View File

@ -0,0 +1,149 @@
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 java.io.File;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.util.TestHarness;
import org.apache.zookeeper.CreateMode;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Base test class for ZooKeeper tests.
*/
public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
static final int TIMEOUT = 10000;
private static final boolean DEBUG = false;
protected static Logger log = LoggerFactory
.getLogger(AbstractZkTestCase.class);
protected ZkTestServer zkServer;
protected String zkDir;
public AbstractZkTestCase() {
}
@BeforeClass
public static void beforeClass() throws Exception {
initCore("solrconfig.xml", "schema12.xml");
}
@Override
public void setUp() throws Exception {
super.setUp();
zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
zkServer = new ZkTestServer(zkDir);
zkServer.run();
System.setProperty("zkHost", zkServer.getZkAddress());
buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(),
getSolrConfigFile(), getSchemaFile());
log.info("####SETUP_START " + getName());
dataDir.mkdirs();
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
CoreContainer.Initializer init = new CoreContainer.Initializer() {
{
this.dataDir = AbstractZkTestCase.dataDir.getAbsolutePath();
}
};
h = new TestHarness("", init);
lrf = h.getRequestFactory("standard", 0, 20, "version", "2.2");
log.info("####SETUP_END " + getName());
}
// static to share with distrib test
static void buildZooKeeper(String zkHost, String zkAddress, String config,
String schema) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.makePath("/solr");
zkClient.close();
zkClient = new SolrZkClient(zkAddress, AbstractZkTestCase.TIMEOUT);
ZkNodeProps props = new ZkNodeProps();
props.put("configName", "conf1");
zkClient.makePath("/collections/collection1", props.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/collection1/shards", CreateMode.PERSISTENT);
zkClient.makePath("/collections/control_collection", props.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/control_collection/shards", CreateMode.PERSISTENT);
putConfig(zkClient, config);
putConfig(zkClient, schema);
putConfig(zkClient, "stopwords.txt");
putConfig(zkClient, "protwords.txt");
putConfig(zkClient, "mapping-ISOLatin1Accent.txt");
putConfig(zkClient, "old_synonyms.txt");
putConfig(zkClient, "synonyms.txt");
zkClient.close();
}
private static void putConfig(SolrZkClient zkConnection, String name)
throws Exception {
zkConnection.setData("/configs/conf1/" + name, new File("solr"
+ File.separator + "conf" + File.separator + name));
}
public void tearDown() throws Exception {
if (DEBUG) {
printLayout(zkServer.getZkHost());
}
zkServer.shutdown();
System.clearProperty("zkHost");
System.clearProperty("solr.test.sys.prop1");
System.clearProperty("solr.test.sys.prop2");
SolrConfig.severeErrors.clear();
super.tearDown();
}
private void printLayout(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.printLayoutToStdOut();
zkClient.close();
}
static void makeSolrZkNode(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(zkHost, TIMEOUT);
zkClient.makePath("/solr");
zkClient.close();
}
}

View File

@ -0,0 +1,283 @@
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 java.net.MalformedURLException;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrServer;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.core.SolrConfig;
import org.junit.BeforeClass;
/**
*
*/
public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
private static final String DEFAULT_COLLECTION = "collection1";
private static final boolean DEBUG = false;
String t1="a_t";
String i1="a_si";
String nint = "n_i";
String tint = "n_ti";
String nfloat = "n_f";
String tfloat = "n_tf";
String ndouble = "n_d";
String tdouble = "n_td";
String nlong = "n_l";
String tlong = "n_tl";
String ndate = "n_dt";
String tdate = "n_tdt";
String oddField="oddField_s";
String missingField="ignore_exception__missing_but_valid_field_t";
String invalidField="ignore_exception__invalid_field_not_in_schema";
public BasicDistributedZkTest() {
fixShardCount = true;
System.setProperty("CLOUD_UPDATE_DELAY", "0");
}
@BeforeClass
public static void beforeClass() throws Exception {
}
@Override
protected void setDistributedParams(ModifiableSolrParams params) {
if (r.nextBoolean()) {
// don't set shards, let that be figured out from the cloud state
params.set("distrib", "true");
} else {
// use shard ids rather than physical locations
StringBuilder sb = new StringBuilder();
for (int i = 0; i < shardCount; i++) {
if (i > 0)
sb.append(',');
sb.append("shard" + (i + 3));
}
params.set("shards", sb.toString());
params.set("distrib", "true");
}
}
@Override
public void doTest() throws Exception {
del("*:*");
indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d);
indexr(id,2, i1, 50 , tlong, 50,t1,"to come to the aid of their country."
);
indexr(id,3, i1, 2, tlong, 2,t1,"how now brown cow"
);
indexr(id,4, i1, -100 ,tlong, 101,t1,"the quick fox jumped over the lazy dog"
);
indexr(id,5, i1, 500, tlong, 500 ,t1,"the quick fox jumped way over the lazy dog"
);
indexr(id,6, i1, -600, tlong, 600 ,t1,"humpty dumpy sat on a wall");
indexr(id,7, i1, 123, tlong, 123 ,t1,"humpty dumpy had a great fall");
indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men");
indexr(id,9, i1, 7, tlong, 7,t1,"couldn't put humpty together again");
indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass");
indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind.");
indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.");
indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out");
indexr(id, 14, "SubjectTerms_mfacet", new String[] {"mathematical models", "mathematical analysis"});
indexr(id, 15, "SubjectTerms_mfacet", new String[] {"test 1", "test 2", "test3"});
indexr(id, 16, "SubjectTerms_mfacet", new String[] {"test 1", "test 2", "test3"});
String[] vals = new String[100];
for (int i=0; i<100; i++) {
vals[i] = "test " + i;
}
indexr(id, 17, "SubjectTerms_mfacet", vals);
for (int i=100; i<150; i++) {
indexr(id, i);
}
commit();
handle.clear();
handle.put("QTime", SKIPVAL);
handle.put("timestamp", SKIPVAL);
// random value sort
for (String f : fieldNames) {
query("q","*:*", "sort",f+" desc");
query("q","*:*", "sort",f+" asc");
}
// these queries should be exactly ordered and scores should exactly match
query("q","*:*", "sort",i1+" desc");
query("q","*:*", "sort",i1+" asc");
query("q","*:*", "sort",i1+" desc", "fl","*,score");
query("q","*:*", "sort",tlong+" asc", "fl","score"); // test legacy behavior - "score"=="*,score"
query("q","*:*", "sort",tlong+" desc");
handle.put("maxScore", SKIPVAL);
query("q","{!func}"+i1);// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
//is agnostic of request params.
handle.remove("maxScore");
query("q","{!func}"+i1, "fl","*,score"); // even scores should match exactly here
handle.put("highlighting", UNORDERED);
handle.put("response", UNORDERED);
handle.put("maxScore", SKIPVAL);
query("q","quick");
query("q","all","fl","id","start","0");
query("q","all","fl","foofoofoo","start","0"); // no fields in returned docs
query("q","all","fl","id","start","100");
handle.put("score", SKIPVAL);
query("q","quick","fl","*,score");
query("q","all","fl","*,score","start","1");
query("q","all","fl","*,score","start","100");
query("q","now their fox sat had put","fl","*,score",
"hl","true","hl.fl",t1);
query("q","now their fox sat had put","fl","foofoofoo",
"hl","true","hl.fl",t1);
query("q","matchesnothing","fl","*,score");
query("q","*:*", "rows",100, "facet","true", "facet.field",t1);
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count");
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2);
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index");
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2);
query("q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1);
query("q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*");
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1);
query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2);
// test faceting multiple things at once
query("q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
,"facet.field",t1);
// test filter tagging, facet exclusion, and naming (multi-select facet support)
query("q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
,"facet.field","{!key=mykey ex=a}"+t1
,"facet.field","{!key=other ex=b}"+t1
,"facet.field","{!key=again ex=a,b}"+t1
,"facet.field",t1
,"fq","{!tag=a}id:[1 TO 7]", "fq","{!tag=b}id:[3 TO 9]"
);
query("q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1");
// test field that is valid in schema but missing in all shards
query("q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2);
// test field that is valid in schema and missing in some shards
query("q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2);
query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1);
/*** TODO: the failure may come back in "exception"
try {
// test error produced for field that is invalid for schema
query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
TestCase.fail("SolrServerException expected for invalid field that is not in schema");
} catch (SolrServerException ex) {
// expected
}
***/
// Try to get better coverage for refinement queries by turning off over requesting.
// This makes it much more likely that we may not get the top facet values and hence
// we turn of that checking.
handle.put("facet_fields", SKIPVAL);
query("q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5);
// check a complex key name
query("q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5);
handle.remove("facet_fields");
// index the same document to two servers and make sure things
// don't blow up.
if (clients.size()>=2) {
index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
for (int i=0; i<clients.size(); i++) {
index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
}
commit();
query("q","duplicate", "hl","true", "hl.fl", t1);
query("q","fox duplicate horses", "hl","true", "hl.fl", t1);
query("q","*:*", "rows",100);
}
// test debugging
handle.put("explain", UNORDERED);
handle.put("debug", UNORDERED);
handle.put("time", SKIPVAL);
query("q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true");
query("q", "id:[1 TO 5]", CommonParams.DEBUG_QUERY, "true");
query("q", "id:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING);
query("q", "id:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS);
query("q", "id:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY);
// TODO: This test currently fails because debug info is obtained only
// on shards with matches.
// query("q","matchesnothing","fl","*,score", "debugQuery", "true");
// Thread.sleep(10000000000L);
if (DEBUG) {
super.printLayout();
}
}
volatile CloudSolrServer solrj;
@Override
protected QueryResponse queryServer(ModifiableSolrParams params) throws SolrServerException {
if (r.nextBoolean())
return super.queryServer(params);
// use the distributed solrj client
if (solrj == null) {
synchronized(this) {
try {
CloudSolrServer server = new CloudSolrServer(zkServer.getZkAddress());
server.setDefaultCollection(DEFAULT_COLLECTION);
solrj = server;
} catch (MalformedURLException e) {
throw new RuntimeException(e);
}
}
}
if (r.nextBoolean())
params.set("collection",DEFAULT_COLLECTION);
QueryResponse rsp = solrj.query(params);
return rsp;
}
@Override
public void tearDown() throws Exception {
super.tearDown();
System.clearProperty("CLOUD_UPDATE_DELAY");
}
}

View File

@ -0,0 +1,129 @@
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.index.LogMergePolicy;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.SolrIndexWriter;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
/**
*
*/
public class BasicZkTest extends AbstractZkTestCase {
@BeforeClass
public static void beforeClass() throws Exception {
initCore("solrconfig.xml", "schema.xml");
}
@Test
public void testBasic() throws Exception {
// test using ZooKeeper
assertTrue("Not using ZooKeeper", h.getCoreContainer().isZooKeeperAware());
ZkController zkController = h.getCoreContainer().getZkController();
// test merge factor picked up
SolrCore core = h.getCore();
SolrIndexWriter writer = new SolrIndexWriter("testWriter", core
.getNewIndexDir(), core.getDirectoryFactory(), false, core.getSchema(),
core.getSolrConfig().mainIndexConfig, core.getDeletionPolicy());
assertEquals("Mergefactor was not picked up", ((LogMergePolicy)writer.getConfig().getMergePolicy()).getMergeFactor(), 8);
writer.close();
lrf.args.put("version", "2.0");
assertQ("test query on empty index", req("qlkciyopsbgzyvkylsjhchghjrdf"),
"//result[@numFound='0']");
// test escaping of ";"
assertU("deleting 42 for no reason at all", delI("42"));
assertU("adding doc#42", adoc("id", "42", "val_s", "aa;bb"));
assertU("does commit work?", commit());
assertQ("backslash escaping semicolon", req("id:42 AND val_s:aa\\;bb"),
"//*[@numFound='1']", "//int[@name='id'][.='42']");
assertQ("quote escaping semicolon", req("id:42 AND val_s:\"aa;bb\""),
"//*[@numFound='1']", "//int[@name='id'][.='42']");
assertQ("no escaping semicolon", req("id:42 AND val_s:aa"),
"//*[@numFound='0']");
assertU(delI("42"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='0']");
// test allowDups default of false
assertU(adoc("id", "42", "val_s", "AAA"));
assertU(adoc("id", "42", "val_s", "BBB"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='BBB']");
assertU(adoc("id", "42", "val_s", "CCC"));
assertU(adoc("id", "42", "val_s", "DDD"));
assertU(commit());
assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='DDD']");
// test deletes
String[] adds = new String[] { add(doc("id", "101"), "allowDups", "false"),
add(doc("id", "101"), "allowDups", "false"),
add(doc("id", "105"), "allowDups", "true"),
add(doc("id", "102"), "allowDups", "false"),
add(doc("id", "103"), "allowDups", "true"),
add(doc("id", "101"), "allowDups", "false"), };
for (String a : adds) {
assertU(a, a);
}
assertU(commit());
zkServer.shutdown();
Thread.sleep(300);
// try a reconnect from disconnect
zkServer = new ZkTestServer(zkDir);
zkServer.run();
// ensure zk still thinks node is up
assertTrue(zkController.getCloudState().liveNodesContain(zkController.getNodeName()));
// test maxint
assertQ(req("q", "id:[100 TO 110]", "rows", "2147483647"),
"//*[@numFound='4']");
// test big limit
assertQ(req("q", "id:[100 TO 111]", "rows", "1147483647"),
"//*[@numFound='4']");
assertQ(req("id:[100 TO 110]"), "//*[@numFound='4']");
assertU(delI("102"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='3']");
assertU(delI("105"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='2']");
assertU(delQ("id:[100 TO 110]"));
assertU(commit());
assertQ(req("id:[100 TO 110]"), "//*[@numFound='0']");
}
}

View File

@ -0,0 +1,248 @@
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 java.io.File;
import java.util.Map;
import java.util.Set;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.CoreContainer.Initializer;
import org.apache.zookeeper.CreateMode;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.Test;
import static org.junit.Assert.*;
/**
* TODO: look at hostPort used below
*/
public class CloudStateUpdateTest extends SolrTestCaseJ4 {
protected static Logger log = LoggerFactory
.getLogger(AbstractZkTestCase.class);
private static final boolean VERBOSE = false;
protected ZkTestServer zkServer;
protected String zkDir;
private CoreContainer container1;
private CoreContainer container2;
private CoreContainer container3;
private File dataDir1;
private File dataDir2;
private File dataDir3;
private File dataDir4;
private Initializer init2;
@BeforeClass
public static void beforeClass() throws Exception {
initCore();
}
@Override
public void setUp() throws Exception {
super.setUp();
System.setProperty("zkClientTimeout", "3000");
zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
zkServer = new ZkTestServer(zkDir);
zkServer.run();
System.setProperty("zkHost", zkServer.getZkAddress());
AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(), zkServer
.getZkAddress(), "solrconfig.xml", "schema.xml");
log.info("####SETUP_START " + getName());
dataDir1 = new File(dataDir + File.separator + "data1");
dataDir1.mkdirs();
dataDir2 = new File(dataDir + File.separator + "data2");
dataDir2.mkdirs();
dataDir3 = new File(dataDir + File.separator + "data3");
dataDir3.mkdirs();
dataDir4 = new File(dataDir + File.separator + "data3");
dataDir4.mkdirs();
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
System.setProperty("hostPort", "1661");
CoreContainer.Initializer init1 = new CoreContainer.Initializer() {
{
this.dataDir = CloudStateUpdateTest.this.dataDir1.getAbsolutePath();
}
};
container1 = init1.initialize();
System.clearProperty("hostPort");
System.setProperty("hostPort", "1662");
init2 = new CoreContainer.Initializer() {
{
this.dataDir = CloudStateUpdateTest.this.dataDir2.getAbsolutePath();
}
};
container2 = init2.initialize();
System.clearProperty("hostPort");
System.setProperty("hostPort", "1663");
CoreContainer.Initializer init3 = new CoreContainer.Initializer() {
{
this.dataDir = CloudStateUpdateTest.this.dataDir3.getAbsolutePath();
}
};
container3 = init3.initialize();
System.clearProperty("hostPort");
log.info("####SETUP_END " + getName());
}
@Test
public void testCoreRegistration() throws Exception {
System.setProperty("CLOUD_UPDATE_DELAY", "1");
ZkNodeProps props2 = new ZkNodeProps();
props2.put("configName", "conf1");
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
zkClient.makePath("/collections/testcore", props2.store(), CreateMode.PERSISTENT);
zkClient.makePath("/collections/testcore/shards", CreateMode.PERSISTENT);
zkClient.close();
CoreDescriptor dcore = new CoreDescriptor(container1, "testcore",
"testcore");
dcore.setDataDir(dataDir4.getAbsolutePath());
SolrCore core = container1.create(dcore);
container1.register(core, false);
// slight pause - TODO: takes an oddly long amount of time to schedule tasks
// with almost no delay ...
Thread.sleep(5000);
ZkController zkController2 = container2.getZkController();
String host = zkController2.getHostName();
CloudState cloudState2 = zkController2.getCloudState();
Map<String,Slice> slices = cloudState2.getSlices("testcore");
assertNotNull(slices);
assertTrue(slices.containsKey(host + ":1661_solr_testcore"));
Slice slice = slices.get(host + ":1661_solr_testcore");
assertEquals(host + ":1661_solr_testcore", slice.getName());
Map<String,ZkNodeProps> shards = slice.getShards();
assertEquals(1, shards.size());
ZkNodeProps zkProps = shards.get(host + ":1661_solr_testcore");
assertNotNull(zkProps);
assertEquals(host + ":1661_solr", zkProps.get("node_name"));
assertEquals("http://" + host + ":1661/solr/testcore", zkProps.get("url"));
Set<String> liveNodes = cloudState2.getLiveNodes();
assertNotNull(liveNodes);
assertEquals(3, liveNodes.size());
container3.shutdown();
// slight pause for watch to trigger
for(int i = 0; i < 4; i++) {
if(zkController2.getCloudState().getLiveNodes().size() == 2) {
break;
}
Thread.sleep(50);
}
assertEquals(2, zkController2.getCloudState().getLiveNodes().size());
// quickly kill / start client
container2.getZkController().getZkClient().getSolrZooKeeper().getConnection()
.disconnect();
container2.shutdown();
container2 = init2.initialize();
Thread.sleep(2000);
if (!container1.getZkController().getCloudState().liveNodesContain(
container2.getZkController().getNodeName())) {
// pause some more
Thread.sleep(5000);
}
assertTrue(container1.getZkController().getCloudState().liveNodesContain(
container2.getZkController().getNodeName()));
}
public void tearDown() throws Exception {
if (VERBOSE) {
printLayout(zkServer.getZkHost());
}
container1.shutdown();
container2.shutdown();
container3.shutdown();
zkServer.shutdown();
super.tearDown();
System.clearProperty("zkClientTimeout");
System.clearProperty("zkHost");
System.clearProperty("hostPort");
System.clearProperty("CLOUD_UPDATE_DELAY");
SolrConfig.severeErrors.clear();
}
private void printLayout(String zkHost) throws Exception {
SolrZkClient zkClient = new SolrZkClient(
zkHost, AbstractZkTestCase.TIMEOUT);
zkClient.printLayoutToStdOut();
zkClient.close();
}
}

View File

@ -0,0 +1,225 @@
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 java.io.File;
import java.io.IOException;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrConfig;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
public class ZkControllerTest extends SolrTestCaseJ4 {
private static final String TEST_NODE_NAME = "test_node_name";
private static final String URL3 = "http://localhost:3133/solr/core1";
private static final String URL2 = "http://localhost:3123/solr/core1";
private static final String SHARD3 = "localhost:3123_solr_core3";
private static final String SHARD2 = "localhost:3123_solr_core2";
private static final String SHARD1 = "localhost:3123_solr_core1";
private static final String COLLECTION_NAME = "collection1";
static final int TIMEOUT = 10000;
private static final String URL1 = "http://localhost:3133/solr/core0";
private static final boolean DEBUG = false;
@BeforeClass
public static void beforeClass() throws Exception {
initCore();
}
@Test
public void testReadShards() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
SolrZkClient zkClient = null;
ZkController zkController = null;
try {
server = new ZkTestServer(zkDir);
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String shardsPath = "/collections/collection1/shards/shardid1";
zkClient.makePath(shardsPath);
addShardToZk(zkClient, shardsPath, SHARD1, URL1);
addShardToZk(zkClient, shardsPath, SHARD2, URL2);
addShardToZk(zkClient, shardsPath, SHARD3, URL3);
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
zkController = new ZkController(server.getZkAddress(),
TIMEOUT, 1000, "localhost", "8983", "solr");
zkController.getZkStateReader().updateCloudState(true);
CloudState cloudInfo = zkController.getCloudState();
Map<String,Slice> slices = cloudInfo.getSlices("collection1");
assertNotNull(slices);
for (Slice slice : slices.values()) {
Map<String,ZkNodeProps> shards = slice.getShards();
if (DEBUG) {
for (String shardName : shards.keySet()) {
ZkNodeProps props = shards.get(shardName);
System.out.println("shard:" + shardName);
System.out.println("props:" + props.toString());
}
}
assertNotNull(shards.get(SHARD1));
assertNotNull(shards.get(SHARD2));
assertNotNull(shards.get(SHARD3));
ZkNodeProps props = shards.get(SHARD1);
assertEquals(URL1, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
props = shards.get(SHARD2);
assertEquals(URL2, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
props = shards.get(SHARD3);
assertEquals(URL3, props.get(ZkStateReader.URL_PROP));
assertEquals(TEST_NODE_NAME, props.get(ZkStateReader.NODE_NAME));
}
} finally {
if (zkClient != null) {
zkClient.close();
}
if (zkController != null) {
zkController.close();
}
if (server != null) {
server.shutdown();
}
}
}
@Test
public void testReadConfigName() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = new ZkTestServer(zkDir);
try {
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String actualConfigName = "firstConfig";
zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName);
ZkNodeProps props = new ZkNodeProps();
props.put("configName", actualConfigName);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + COLLECTION_NAME , props.store(), CreateMode.PERSISTENT);
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
zkClient.close();
ZkController zkController = new ZkController(server.getZkAddress(), TIMEOUT, 1000,
"localhost", "8983", "/solr");
try {
String configName = zkController.readConfigName(COLLECTION_NAME);
assertEquals(configName, actualConfigName);
} finally {
zkController.close();
}
} finally {
server.shutdown();
}
}
@Test
public void testUploadToCloud() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = new ZkTestServer(zkDir);
ZkController zkController = null;
try {
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkController = new ZkController(server.getZkAddress(),
TIMEOUT, 1000, "localhost", "8983", "/solr");
zkController.uploadToZK(new File("solr/conf"),
ZkController.CONFIGS_ZKNODE + "/config1");
if (DEBUG) {
zkController.printLayoutToStdOut();
}
} finally {
if (zkController != null) {
zkController.close();
}
server.shutdown();
}
}
private void addShardToZk(SolrZkClient zkClient, String shardsPath,
String zkNodeName, String url) throws IOException,
KeeperException, InterruptedException {
ZkNodeProps props = new ZkNodeProps();
props.put(ZkStateReader.URL_PROP, url);
props.put(ZkStateReader.NODE_NAME, TEST_NODE_NAME);
byte[] bytes = props.store();
zkClient
.create(shardsPath + "/" + zkNodeName, bytes, CreateMode.PERSISTENT);
}
public void tearDown() throws Exception {
SolrConfig.severeErrors.clear();
super.tearDown();
}
}

View File

@ -0,0 +1,51 @@
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 java.io.IOException;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.junit.Test;
import static org.junit.Assert.*;
public class ZkNodePropsTest extends SolrTestCaseJ4 {
@Test
public void testBasic() throws IOException {
ZkNodeProps props = new ZkNodeProps();
props.put("prop1", "value1");
props.put("prop2", "value2");
props.put("prop3", "value3");
props.put("prop4", "value4");
props.put("prop5", "value5");
props.put("prop6", "value6");
byte[] bytes = props.store();
ZkNodeProps props2 = new ZkNodeProps();
props2.load(bytes);
assertEquals("value1", props2.get("prop1"));
assertEquals("value2", props2.get("prop2"));
assertEquals("value3", props2.get("prop3"));
assertEquals("value4", props2.get("prop4"));
assertEquals("value5", props2.get("prop5"));
assertEquals("value6", props2.get("prop6"));
}
}

View File

@ -0,0 +1,240 @@
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 java.io.File;
import java.util.concurrent.atomic.AtomicInteger;
import junit.framework.TestCase;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
public class ZkSolrClientTest extends AbstractSolrTestCase {
private static final boolean DEBUG = false;
public void testConnect() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
server = new ZkTestServer(zkDir);
server.run();
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), 100);
zkClient.close();
server.shutdown();
}
public void testMakeRootNode() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
server = new ZkTestServer(zkDir);
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
SolrZkClient zkClient = new SolrZkClient(server.getZkHost(),
AbstractZkTestCase.TIMEOUT);
assertTrue(zkClient.exists("/solr"));
zkClient.close();
server.shutdown();
}
public void testReconnect() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
ZkTestServer server = null;
SolrZkClient zkClient = null;
try {
server = new ZkTestServer(zkDir);
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
zkClient = new SolrZkClient(server.getZkAddress(), AbstractZkTestCase.TIMEOUT);
String shardsPath = "/collections/collection1/shards";
zkClient.makePath(shardsPath);
zkClient.makePath("collections/collection1");
int zkServerPort = server.getPort();
// this tests disconnect state
server.shutdown();
Thread.sleep(80);
try {
zkClient.makePath("collections/collection2");
TestCase.fail("Server should be down here");
} catch (KeeperException.ConnectionLossException e) {
}
// bring server back up
server = new ZkTestServer(zkDir, zkServerPort);
server.run();
// TODO: can we do better?
// wait for reconnect
Thread.sleep(600);
try {
zkClient.makePath("collections/collection3");
} catch (KeeperException.ConnectionLossException e) {
Thread.sleep(5000); // try again in a bit
zkClient.makePath("collections/collection3");
}
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
assertNotNull(zkClient.exists("/collections/collection3", null));
assertNotNull(zkClient.exists("/collections/collection1", null));
// simulate session expiration
// one option
long sessionId = zkClient.getSolrZooKeeper().getSessionId();
server.expire(sessionId);
// another option
//zkClient.getSolrZooKeeper().getConnection().disconnect();
// this tests expired state
Thread.sleep(1000); // pause for reconnect
for (int i = 0; i < 8; i++) {
try {
zkClient.makePath("collections/collection4");
break;
} catch (KeeperException.SessionExpiredException e) {
} catch (KeeperException.ConnectionLossException e) {
}
Thread.sleep(1000 * i);
}
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
assertNotNull("Node does not exist, but it should", zkClient.exists("/collections/collection4", null));
} finally {
if (zkClient != null) {
zkClient.close();
}
if (server != null) {
server.shutdown();
}
}
}
public void testWatchChildren() throws Exception {
String zkDir = dataDir.getAbsolutePath() + File.separator
+ "zookeeper/server1/data";
final AtomicInteger cnt = new AtomicInteger();
ZkTestServer server = new ZkTestServer(zkDir);
server.run();
Thread.sleep(400);
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
final SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), AbstractZkTestCase.TIMEOUT);
try {
zkClient.makePath("/collections");
zkClient.getChildren("/collections", new Watcher() {
public void process(WatchedEvent event) {
if (DEBUG) {
System.out.println("children changed");
}
cnt.incrementAndGet();
// remake watch
try {
zkClient.getChildren("/collections", this);
} catch (KeeperException e) {
throw new RuntimeException(e);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
});
zkClient.makePath("/collections/collection99/shards");
zkClient.makePath("collections/collection99/config=collection1");
zkClient.makePath("collections/collection99/config=collection3");
zkClient.makePath("/collections/collection97/shards");
if (DEBUG) {
zkClient.printLayoutToStdOut();
}
// pause for the watches to fire
Thread.sleep(700);
if (cnt.intValue() < 2) {
Thread.sleep(4000); // wait a bit more
}
assertEquals(2, cnt.intValue());
} finally {
if (zkClient != null) {
zkClient.close();
}
if (server != null) {
server.shutdown();
}
}
}
@Override
public String getSchemaFile() {
return null;
}
@Override
public String getSolrConfigFile() {
return null;
}
public void tearDown() throws Exception {
SolrConfig.severeErrors.clear();
super.tearDown();
}
}

View File

@ -0,0 +1,319 @@
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 java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.List;
import javax.management.JMException;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.zookeeper.jmx.ManagedUtil;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.SessionTracker.Session;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
public class ZkTestServer {
protected final ZKServerMain zkServer = new ZKServerMain();
private String zkDir;
private int clientPort;
private Thread zooThread;
class ZKServerMain {
private NIOServerCnxn.Factory cnxnFactory;
private ZooKeeperServer zooKeeperServer;
protected void initializeAndRun(String[] args) throws ConfigException,
IOException {
try {
ManagedUtil.registerLog4jMBeans();
} catch (JMException e) {
}
ServerConfig config = new ServerConfig();
if (args.length == 1) {
config.parse(args[0]);
} else {
config.parse(args);
}
runFromConfig(config);
}
/**
* Run from a ServerConfig.
*
* @param config ServerConfig to use.
* @throws IOException
*/
public void runFromConfig(ServerConfig config) throws IOException {
try {
// Note that this thread isn't going to be doing anything else,
// so rather than spawning another thread, we will just call
// run() in this thread.
// create a file logger url from the command line args
zooKeeperServer = new ZooKeeperServer();
FileTxnSnapLog ftxn = new FileTxnSnapLog(new File(config
.getDataLogDir()), new File(config.getDataDir()));
zooKeeperServer.setTxnLogFactory(ftxn);
zooKeeperServer.setTickTime(config.getTickTime());
cnxnFactory = new NIOServerCnxn.Factory(config.getClientPortAddress(), config
.getMaxClientCnxns());
cnxnFactory.startup(zooKeeperServer);
cnxnFactory.join();
if (zooKeeperServer.isRunning()) {
zooKeeperServer.shutdown();
}
} catch (InterruptedException e) {
}
}
/**
* Shutdown the serving instance
* @throws IOException
*/
protected void shutdown() throws IOException {
zooKeeperServer.shutdown();
zooKeeperServer.getZKDatabase().close();
waitForServerDown(getZkHost() + ":" + getPort(), 5000);
cnxnFactory.shutdown();
}
public int getLocalPort() {
if (cnxnFactory == null) {
throw new IllegalStateException("A port has not yet been selected");
}
int port = cnxnFactory.getLocalPort();
if (port == 0) {
throw new IllegalStateException("A port has not yet been selected");
}
return port;
}
}
public ZkTestServer(String zkDir) {
this.zkDir = zkDir;
}
public ZkTestServer(String zkDir, int port) {
this.zkDir = zkDir;
this.clientPort = port;
}
public String getZkHost() {
return "127.0.0.1:" + zkServer.getLocalPort();
}
public String getZkAddress() {
return "127.0.0.1:" + zkServer.getLocalPort() + "/solr";
}
public int getPort() {
return zkServer.getLocalPort();
}
public void expire(final long sessionId) {
zkServer.zooKeeperServer.expire(new Session() {
@Override
public long getSessionId() {
return sessionId;
}
@Override
public int getTimeout() {
return 4000;
}});
}
public void run() throws InterruptedException {
// we don't call super.setUp
zooThread = new Thread() {
@Override
public void run() {
ServerConfig config = new ServerConfig() {
{
setClientPort(ZkTestServer.this.clientPort);
this.dataDir = zkDir;
this.dataLogDir = zkDir;
this.tickTime = 1500;
}
public void setClientPort(int clientPort) {
if (clientPortAddress != null) {
try {
this.clientPortAddress = new InetSocketAddress(
InetAddress.getByName(clientPortAddress.getHostName()), clientPort);
} catch (UnknownHostException e) {
throw new RuntimeException(e);
}
} else {
this.clientPortAddress = new InetSocketAddress(clientPort);
}
}
};
try {
zkServer.runFromConfig(config);
} catch (Throwable e) {
throw new RuntimeException(e);
}
}
};
zooThread.setDaemon(true);
zooThread.start();
int cnt = 0;
int port = -1;
try {
port = getPort();
} catch(IllegalStateException e) {
}
while (port < 1) {
Thread.sleep(100);
try {
port = getPort();
} catch(IllegalStateException e) {
}
if (cnt == 40) {
throw new RuntimeException("Could not get the port for ZooKeeper server");
}
cnt++;
}
}
@SuppressWarnings("deprecation")
public void shutdown() throws IOException {
SolrTestCaseJ4.ignoreException("java.nio.channels.ClosedChannelException");
// TODO: this can log an exception while trying to unregister a JMX MBean
try {
zkServer.shutdown();
} finally {
SolrTestCaseJ4.resetExceptionIgnores();
}
}
public static boolean waitForServerDown(String hp, long timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
HostPort hpobj = parseHostPortList(hp).get(0);
send4LetterWord(hpobj.host, hpobj.port, "stat");
} catch (IOException e) {
return true;
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
}
public static class HostPort {
String host;
int port;
HostPort(String host, int port) {
this.host = host;
this.port = port;
}
}
/**
* Send the 4letterword
* @param host the destination host
* @param port the destination port
* @param cmd the 4letterword
* @return
* @throws IOException
*/
public static String send4LetterWord(String host, int port, String cmd)
throws IOException
{
Socket sock = new Socket(host, port);
BufferedReader reader = null;
try {
OutputStream outstream = sock.getOutputStream();
outstream.write(cmd.getBytes());
outstream.flush();
// this replicates NC - close the output stream before reading
sock.shutdownOutput();
reader =
new BufferedReader(
new InputStreamReader(sock.getInputStream()));
StringBuilder sb = new StringBuilder();
String line;
while((line = reader.readLine()) != null) {
sb.append(line + "\n");
}
return sb.toString();
} finally {
sock.close();
if (reader != null) {
reader.close();
}
}
}
public static List<HostPort> parseHostPortList(String hplist) {
ArrayList<HostPort> alist = new ArrayList<HostPort>();
for (String hp : hplist.split(",")) {
int idx = hp.lastIndexOf(':');
String host = hp.substring(0, idx);
int port;
try {
port = Integer.parseInt(hp.substring(idx + 1));
} catch (RuntimeException e) {
throw new RuntimeException("Problem parsing " + hp + e.toString());
}
alist.add(new HostPort(host, port));
}
return alist;
}
}

View File

@ -1,6 +1,7 @@
package org.apache.solr.handler.component;
import org.apache.solr.BaseDistributedSearchTestCase;
import org.junit.Test;
/**
* Test for TermsComponent distributed querying
@ -12,6 +13,7 @@ public class DistributedTermsComponentTest extends BaseDistributedSearchTestCase
@Override
public void doTest() throws Exception {
del("*:*");
index(id, 18, "b_t", "snake spider shark snail slug seal");
index(id, 19, "b_t", "snake spider shark snail slug");
index(id, 20, "b_t", "snake spider shark snail");

View File

@ -0,0 +1,34 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!--
All (relative) paths are relative to the installation path
persistent: Save changes made via the API to this file
sharedLib: path to a lib directory that will be shared across all cores
-->
<solr persistent="false">
<!--
adminPath: RequestHandler path to manage cores.
If 'null' (or absent), cores will not be manageable via request handler
-->
<cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="8000">
<core name="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" instanceDir="."/>
</cores>
</solr>

View File

@ -18,8 +18,6 @@
package org.apache.solr.client.solrj.embedded;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.util.Random;
import javax.servlet.http.HttpServlet;
@ -27,9 +25,11 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.mortbay.component.LifeCycle;
import org.mortbay.jetty.Connector;
import org.mortbay.jetty.Handler;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.Connector;
import org.mortbay.jetty.nio.SelectChannelConnector;
import org.mortbay.jetty.nio.SelectChannelConnector;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.FilterHolder;
@ -41,43 +41,30 @@ import org.mortbay.log.Logger;
*
* @since solr 1.3
*/
public class JettySolrRunner
{
public class JettySolrRunner {
Server server;
FilterHolder dispatchFilter;
String context;
public JettySolrRunner( String context, int port )
{
this.init( context, port );
private String solrConfigFilename;
private boolean waitOnSolr = false;
public JettySolrRunner(String context, int port) {
this.init(context, port);
}
public JettySolrRunner( String context, int port, String solrConfigFilename )
{
this.init( context, port );
if (solrConfigFilename != null)
dispatchFilter.setInitParameter("solrconfig-filename", solrConfigFilename);
public JettySolrRunner(String context, int port, String solrConfigFilename) {
this.init(context, port);
this.solrConfigFilename = solrConfigFilename;
}
// public JettySolrRunner( String context, String home, String dataDir, int port, boolean log )
// {
// if(!log) {
// System.setProperty("org.mortbay.log.class", NoLog.class.getName() );
// System.setProperty("java.util.logging.config.file", home+"/conf/logging.properties");
// NoLog noLogger = new NoLog();
// org.mortbay.log.Log.setLog(noLogger);
// }
//
// // Initalize JNDI
// Config.setInstanceDir(home);
// new SolrCore(dataDir, new IndexSchema(home+"/conf/schema.xml"));
// this.init( context, port );
// }
private void init( String context, int port )
{
private void init(String context, int port) {
this.context = context;
server = new Server( port );
server = new Server(port);
server.setStopAtShutdown(true);
if (System.getProperty("jetty.testMode") != null) {
SelectChannelConnector connector = new SelectChannelConnector();
connector.setPort(port);
@ -85,72 +72,76 @@ public class JettySolrRunner
server.setConnectors(new Connector[] { connector });
server.setSessionIdManager(new HashSessionIdManager(new Random()));
}
server.setStopAtShutdown( true );
// Initialize the servlets
Context root = new Context( server, context, Context.SESSIONS );
final Context root = new Context(server, context, Context.SESSIONS);
server.addLifeCycleListener(new LifeCycle.Listener() {
public void lifeCycleStopping(LifeCycle arg0) {
System.clearProperty("hostPort");
}
public void lifeCycleStopped(LifeCycle arg0) {}
public void lifeCycleStarting(LifeCycle arg0) {
synchronized (JettySolrRunner.this) {
waitOnSolr = true;
JettySolrRunner.this.notify();
}
}
public void lifeCycleStarted(LifeCycle arg0) {
System.setProperty("hostPort", Integer.toString(getLocalPort()));
if (solrConfigFilename != null)
System.setProperty("solrconfig", solrConfigFilename);
dispatchFilter = root.addFilter(SolrDispatchFilter.class, "*",
Handler.REQUEST);
if (solrConfigFilename != null)
System.clearProperty("solrconfig");
}
public void lifeCycleFailure(LifeCycle arg0, Throwable arg1) {
System.clearProperty("hostPort");
}
});
// for some reason, there must be a servlet for this to get applied
root.addServlet( Servlet404.class, "/*" );
dispatchFilter = root.addFilter( SolrDispatchFilter.class, "*", Handler.REQUEST );
root.addServlet(Servlet404.class, "/*");
}
//------------------------------------------------------------------------------------------------
//------------------------------------------------------------------------------------------------
public void start() throws Exception
{
// ------------------------------------------------------------------------------------------------
// ------------------------------------------------------------------------------------------------
public void start() throws Exception {
start(true);
}
public void start(boolean waitForSolr) throws Exception
{
if(!server.isRunning() ) {
public void start(boolean waitForSolr) throws Exception {
if (!server.isRunning()) {
server.start();
}
if (waitForSolr) waitForSolr(context);
synchronized (JettySolrRunner.this) {
int cnt = 0;
while (!waitOnSolr) {
this.wait(100);
if (cnt++ == 5) {
throw new RuntimeException("Jetty/Solr unresponsive");
}
}
}
}
public void stop() throws Exception
{
if( server.isRunning() ) {
public void stop() throws Exception {
if (server.isRunning()) {
server.stop();
server.join();
}
}
/** Waits until a ping query to the solr server succeeds,
* retrying every 200 milliseconds up to 2 minutes.
*/
public void waitForSolr(String context) throws Exception
{
int port = getLocalPort();
// A raw term query type doesn't check the schema
URL url = new URL("http://localhost:"+port+context+"/select?q={!raw+f=junit_test_query}ping");
Exception ex = null;
// Wait for a total of 20 seconds: 100 tries, 200 milliseconds each
for (int i=0; i<600; i++) {
try {
InputStream stream = url.openStream();
stream.close();
} catch (IOException e) {
// e.printStackTrace();
ex = e;
Thread.sleep(200);
continue;
}
return;
}
throw new RuntimeException("Jetty/Solr unresponsive",ex);
}
/**
* Returns the Local Port of the first Connector found for the jetty Server.
*
* @exception RuntimeException if there is no Connector
*/
public int getLocalPort() {
@ -161,80 +152,78 @@ public class JettySolrRunner
return conns[0].getLocalPort();
}
//--------------------------------------------------------------
//--------------------------------------------------------------
/**
// --------------------------------------------------------------
// --------------------------------------------------------------
/**
* This is a stupid hack to give jetty something to attach to
*/
public static class Servlet404 extends HttpServlet
{
public static class Servlet404 extends HttpServlet {
@Override
public void service(HttpServletRequest req, HttpServletResponse res ) throws IOException
{
res.sendError( 404, "Can not find: "+req.getRequestURI() );
public void service(HttpServletRequest req, HttpServletResponse res)
throws IOException {
res.sendError(404, "Can not find: " + req.getRequestURI());
}
}
/**
* A main class that starts jetty+solr
* This is useful for debugging
* A main class that starts jetty+solr This is useful for debugging
*/
public static void main( String[] args )
{
public static void main(String[] args) {
try {
JettySolrRunner jetty = new JettySolrRunner( "/solr", 3456 );
JettySolrRunner jetty = new JettySolrRunner("/solr", 3456);
jetty.start();
}
catch( Exception ex ) {
} catch (Exception ex) {
ex.printStackTrace();
}
}
}
class NoLog implements Logger {
private static boolean debug = System.getProperty("DEBUG", null) != null;
class NoLog implements Logger
{
private static boolean debug = System.getProperty("DEBUG",null)!=null;
private final String name;
public NoLog()
{
public NoLog() {
this(null);
}
public NoLog(String name)
{
this.name=name==null?"":name;
public NoLog(String name) {
this.name = name == null ? "" : name;
}
public boolean isDebugEnabled()
{
public boolean isDebugEnabled() {
return debug;
}
public void setDebugEnabled(boolean enabled)
{
debug=enabled;
}
public void info(String msg,Object arg0, Object arg1) {}
public void debug(String msg,Throwable th){}
public void debug(String msg,Object arg0, Object arg1){}
public void warn(String msg,Object arg0, Object arg1){}
public void warn(String msg, Throwable th){}
public Logger getLogger(String name)
{
if ((name==null && this.name==null) ||
(name!=null && name.equals(this.name)))
public void setDebugEnabled(boolean enabled) {
debug = enabled;
}
public void info(String msg, Object arg0, Object arg1) {
}
public void debug(String msg, Throwable th) {
}
public void debug(String msg, Object arg0, Object arg1) {
}
public void warn(String msg, Object arg0, Object arg1) {
}
public void warn(String msg, Throwable th) {
}
public Logger getLogger(String name) {
if ((name == null && this.name == null)
|| (name != null && name.equals(this.name)))
return this;
return new NoLog(name);
}
@Override
public String toString()
{
return "NOLOG["+name+"]";
public String toString() {
return "NOLOG[" + name + "]";
}
}

View File

@ -50,6 +50,7 @@
[<a href="stats.jsp">Statistics</a>]
[<a href="registry.jsp">Info</a>]
[<a href="distributiondump.jsp">Distribution</a>]
[<a href="zookeeper.jsp">ZooKeeper</a>]
[<a href="ping">Ping</a>]
[<a href="logging">Logging</a>]
</td>

View File

@ -0,0 +1,478 @@
<%@ page contentType="text/html; charset=utf-8" pageEncoding="UTF-8"%>
<%--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
--%>
<%@ page import="javax.servlet.jsp.JspWriter,java.io.IOException,org.apache.zookeeper.*,org.apache.zookeeper.data.Stat,org.apache.solr.core.*,org.apache.solr.cloud.*,org.apache.solr.common.cloud.*,org.apache.solr.common.util.*,java.util.concurrent.TimeoutException"%>
<%@ page import="java.io.*"%>
<%@ page import="java.util.*"%>
<%@ page import="java.net.URLEncoder"%>
<%@include file="header.jsp" %>
<br clear="all">
<h2>Zookeeper Browser</h2>
<%
String path = request.getParameter("path");
String addr = request.getParameter("addr");
if (addr != null && addr.length() == 0)
addr = null;
String detailS = request.getParameter("detail");
boolean detail = detailS != null && detailS.equals("true");
ZKPrinter printer = new ZKPrinter(out, core, addr);
printer.detail = detail;
String tryAddr = printer.keeperAddr != null ? printer.keeperAddr
: "localhost:2181";
%>
<form method="GET" action="zookeeper.jsp" accept-charset="UTF-8">
<table>
<tr>
<td>
<strong> <%
XML.escapeCharData(printer.zkClient == null ? "Disconnected"
: ("Connected to zookeeper " + printer.keeperAddr), out);
%> </strong>
</td>
<td>
Connect to different zookeeper:
<input class="std" name="addr" type="text" value="<%XML.escapeCharData(tryAddr, out);%>">
</td>
<td>
<input class="stdbutton" type="submit" value="CONNECT">
</td>
</tr>
<tr>
</table>
</form>
<%
try {
printer.print(path);
} finally {
printer.close();
}
%>
</body>
</html>
<%!static class ZKPrinter {
static boolean FULLPATH_DEFAULT = false;
boolean indent = true;
boolean fullpath = FULLPATH_DEFAULT;
boolean detail = false;
String addr; // the address passed to us
String keeperAddr; // the address we're connected to
SolrZkClient zkClient;
boolean doClose; // close the client after done if we opened it
JspWriter out;
int level;
int maxData = 100;
private boolean levelchange;
public ZKPrinter(JspWriter out, SolrCore core, String addr)
throws IOException {
this.out = out;
this.addr = addr;
if (addr == null) {
ZkController controller = core.getCoreDescriptor().getCoreContainer().getZkController();
if (controller != null) {
// this core is zk enabled
keeperAddr = controller.getZkServerAddress();
zkClient = controller.getZkClient();
if (zkClient != null && zkClient.isConnected()) {
return;
} else {
// try a different client with this address
addr = keeperAddr;
}
}
}
keeperAddr = addr;
if (addr == null) {
out.println("Zookeeper is not configured for this Solr Core. Please try connecting to an alternate zookeeper address.");
return;
}
try {
zkClient = new SolrZkClient(addr, 10000);
doClose = true;
} catch (TimeoutException e) {
out.println("Could not connect to zookeeper at " + addr);
zkClient = null;
return;
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
out.println("Could not connect to zookeeper at " + addr);
zkClient = null;
return;
}
}
public void close() {
try {
if (doClose) zkClient.close();
} catch (InterruptedException e) {
// ignore exception on close
}
}
// main entry point
void print(String path) throws IOException {
if (zkClient == null)
return;
out.print("<table>");
out.print("<tr><td>");
out.print("[");
url("ROOT", "/", false);
out.print("]");
// normalize path
if (path == null)
path = "/";
else {
path.trim();
if (path.length() == 0)
path = "/";
}
if (path.endsWith("/") && path.length() > 1) {
path = path.substring(0, path.length() - 1);
}
int idx = path.lastIndexOf('/');
String parent = idx >= 0 ? path.substring(0, idx) : path;
if (parent.length() == 0)
parent = "/";
out.print(" [");
url("PARENT", parent, detail);
out.print("]");
out.print("</td></tr>");
if (detail) {
out.print("<tr><td>");
printZnode(path);
out.print("</td></tr>");
}
out.print("<tr><td>");
printTree(path);
out.print("</td></tr>");
out.print("</table>");
}
void exception(Exception e) {
try {
out.println(e.toString());
} catch (IOException e1) {
// nothing we can do
}
}
void xmlescape(String s) {
try {
XML.escapeCharData(s, out);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
void up() throws IOException {
level++;
levelchange = true;
}
void down() throws IOException {
level--;
levelchange = true;
}
void indent() throws IOException {
// if we are using blockquote and just changed indent levels, don't output a break
// if (fullpath || !levelchange)
out.println("<br>");
levelchange = false;
for (int i=0; i<level; i++)
out.println("&nbsp;&nbsp;&nbsp;&nbsp;");
// if fullpath, no indent is needed
// if not, we are currently using blockquote which the browser
// will take care of indenting.
}
// collapse all whitespace to a single space or escaped newline
String compress(String str) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < str.length(); i++) {
char ch = str.charAt(i);
boolean whitespace = false;
boolean newline = false;
while (Character.isWhitespace(ch)) {
whitespace = true;
if (ch == '\n')
newline = true;
if (++i >= str.length())
return sb.toString();
ch = str.charAt(i);
}
if (newline) {
// sb.append("\\n");
sb.append(" "); // collapse newline to two spaces
} else if (whitespace) {
sb.append(' ');
}
// TODO: handle non-printable chars
sb.append(ch);
if (sb.length() >= maxData)
return sb.toString() + "...";
}
return sb.toString();
}
void url(String label, String path, boolean detail) throws IOException {
try {
out.print("<a href=\"zookeeper.jsp?");
if (path != null) {
out.print("path=");
out.print(URLEncoder.encode(path, "UTF-8"));
}
if (detail) {
out.print("&detail=" + detail);
}
if (fullpath != FULLPATH_DEFAULT) {
out.print("&fullpath=" + fullpath);
}
if (addr != null) {
out.print("&addr=");
out.print(URLEncoder.encode(addr, "UTF-8"));
}
out.print("\">");
xmlescape(label);
out.print("</a>");
} catch (UnsupportedEncodingException e) {
exception(e);
}
}
void printTree(String path) throws IOException {
indent();
// TODO: make a link from the path
String label = path;
if (!fullpath) {
int idx = path.lastIndexOf('/');
label = idx > 0 ? path.substring(idx + 1) : path;
}
url(label, path, true);
out.print(" (");
Stat stat = new Stat();
try {
byte[] data = zkClient.getData(path, null, stat);
if (stat.getEphemeralOwner() != 0)
out.print("ephemeral ");
out.print("v=" + stat.getVersion());
if (stat.getNumChildren() != 0) {
out.print(" children=" + stat.getNumChildren());
}
out.print(")");
if (data != null) {
String str;
try {
str = new String(data, "UTF-8");
out.print(" \"");
xmlescape(compress(str));
out.print("\"");
} catch (UnsupportedEncodingException e) {
// not UTF8
StringBuilder sb = new StringBuilder("BIN(");
sb.append("len=" + data.length);
sb.append("hex=");
int limit = Math.min(data.length, maxData / 2);
for (int i = 0; i < limit; i++) {
byte b = data[i];
sb.append(StrUtils.HEX_DIGITS[(b >> 4) & 0xf]);
sb.append(StrUtils.HEX_DIGITS[b & 0xf]);
}
if (limit != data.length)
sb.append("...");
sb.append(")");
str = sb.toString();
out.print(str);
}
}
} catch (IllegalArgumentException e) {
// path doesn't exist (must have been removed)
out.println("(path gone)");
} catch (KeeperException e) {
e.printStackTrace();
} catch (InterruptedException e) {
e.printStackTrace();
}
if (stat.getNumChildren() <= 0)
return;
List<String> children = null;
try {
children = zkClient.getChildren(path, null);
} catch (KeeperException e) {
exception(e);
return;
} catch (InterruptedException e) {
exception(e);
} catch (IllegalArgumentException e) {
// path doesn't exist (must have been removed)
out.println("(children gone)");
}
up();
for (String child : children) {
String childPath = path + (path.endsWith("/") ? "" : "/") + child;
printTree(childPath);
}
down();
}
String time(long ms) {
return (new Date(ms)).toString() + " (" + ms + ")";
}
void printZnode(String path) throws IOException {
try {
Stat stat = new Stat();
byte[] data = zkClient.getData(path, null, stat);
out.print("<h2>");
xmlescape(path);
out.print("</h2>");
up();
indent();
out.print("version = " + stat.getVersion());
indent();
out.print("aversion = " + stat.getAversion());
indent();
out.print("cversion = " + stat.getCversion());
indent();
out.print("ctime = " + time(stat.getCtime()));
indent();
out.print("mtime = " + time(stat.getMtime()));
indent();
out.print("czxid = " + stat.getCzxid());
indent();
out.print("mzxid = " + stat.getMzxid());
indent();
out.print("pzxid = " + stat.getPzxid());
indent();
out.print("numChildren = " + stat.getNumChildren());
indent();
out.print("ephemeralOwner = " + stat.getEphemeralOwner());
indent();
out.print("dataLength = " + stat.getDataLength());
if (data != null) {
boolean isBinary = false;
String str;
try {
str = new String(data, "UTF-8");
} catch (UnsupportedEncodingException e) {
// The results are unspecified
// when the bytes are not properly encoded.
// not UTF8
StringBuilder sb = new StringBuilder(data.length * 2);
for (int i = 0; i < data.length; i++) {
byte b = data[i];
sb.append(StrUtils.HEX_DIGITS[(b >> 4) & 0xf]);
sb.append(StrUtils.HEX_DIGITS[b & 0xf]);
if ((i & 0x3f) == 0x3f)
sb.append("\n");
}
str = sb.toString();
}
int nLines = 1;
int lineLen = 0;
int maxLineLen = 10; // the minimum
for (int i = 0; i < str.length(); i++) {
if (str.charAt(i) == '\n') {
nLines++;
maxLineLen = Math.max(maxLineLen, lineLen);
lineLen = 0;
} else {
lineLen++;
}
}
indent();
out.println("<form method='post' action=''>");
out.println("<textarea class='big' wrap='off' readonly rows='"
+ Math.min(20, nLines)
// + "' cols='" + Math.min(80, maxLineLen+1)
// + "' cols='" + (maxLineLen+1)
+ "' name='data'>");
xmlescape(str);
out.println("</textarea></form>");
}
down();
} catch (KeeperException e) {
exception(e);
return;
} catch (InterruptedException e) {
exception(e);
}
}
}%>