From b55f338757d2bbd616bc5f77c3a05dcfd20b3022 Mon Sep 17 00:00:00 2001 From: Mark Robert Miller Date: Wed, 13 Oct 2010 17:01:13 +0000 Subject: [PATCH] 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 --- solr/CHANGES.txt | 4 + solr/lib/log4j-over-slf4j-1.5.5.jar | 2 + solr/lib/zookeeper-3.3.1.jar | 2 + .../apache/solr/common/cloud/CloudState.java | 144 ++++ .../solr/common/cloud/ConnectionManager.java | 140 ++++ .../cloud/DefaultConnectionStrategy.java | 74 ++ .../apache/solr/common/cloud/OnReconnect.java | 22 + .../org/apache/solr/common/cloud/Slice.java | 41 ++ .../solr/common/cloud/SolrZkClient.java | 493 +++++++++++++ .../solr/common/cloud/SolrZooKeeper.java | 37 + .../cloud/ZkClientConnectionStrategy.java | 36 + .../apache/solr/common/cloud/ZkNodeProps.java | 58 ++ .../solr/common/cloud/ZkStateReader.java | 400 +++++++++++ .../solr/common/cloud/ZooKeeperException.java | 33 + .../solr/common/params/CoreAdminParams.java | 6 + .../apache/solr/cloud/CloudDescriptor.java | 52 ++ .../org/apache/solr/cloud/SolrZkServer.java | 477 +++++++++++++ .../org/apache/solr/cloud/ZkController.java | 659 ++++++++++++++++++ .../solr/cloud/ZkSolrResourceLoader.java | 120 ++++ .../org/apache/solr/core/CoreContainer.java | 339 +++++++-- .../org/apache/solr/core/CoreDescriptor.java | 24 + .../java/org/apache/solr/core/SolrCore.java | 24 +- .../apache/solr/core/SolrResourceLoader.java | 11 +- .../solr/handler/admin/CoreAdminHandler.java | 23 +- .../handler/component/QueryComponent.java | 120 +++- .../component/QueryElevationComponent.java | 37 +- .../handler/component/ResponseBuilder.java | 13 +- .../solr/handler/component/SearchHandler.java | 132 +++- .../solr/handler/component/ShardRequest.java | 2 - .../solr/handler/component/ShardResponse.java | 5 + .../handler/component/TermsComponent.java | 9 +- .../client/solrj/impl/CloudSolrServer.java | 148 ++++ .../client/solrj/impl/LBHttpSolrServer.java | 407 ++++++++--- .../solr/BaseDistributedSearchTestCase.java | 105 ++- .../cloud/AbstractDistributedZkTestCase.java | 92 +++ .../apache/solr/cloud/AbstractZkTestCase.java | 149 ++++ .../solr/cloud/BasicDistributedZkTest.java | 283 ++++++++ .../org/apache/solr/cloud/BasicZkTest.java | 129 ++++ .../solr/cloud/CloudStateUpdateTest.java | 248 +++++++ .../apache/solr/cloud/ZkControllerTest.java | 225 ++++++ .../apache/solr/cloud/ZkNodePropsTest.java | 51 ++ .../apache/solr/cloud/ZkSolrClientTest.java | 240 +++++++ .../org/apache/solr/cloud/ZkTestServer.java | 319 +++++++++ .../DistributedTermsComponentTest.java | 2 + solr/src/test/test-files/solr/solr.xml | 34 + .../solrj/embedded/JettySolrRunner.java | 249 ++++--- solr/src/webapp/web/admin/index.jsp | 1 + solr/src/webapp/web/admin/zookeeper.jsp | 478 +++++++++++++ 48 files changed, 6328 insertions(+), 371 deletions(-) create mode 100644 solr/lib/log4j-over-slf4j-1.5.5.jar create mode 100644 solr/lib/zookeeper-3.3.1.jar create mode 100644 solr/src/common/org/apache/solr/common/cloud/CloudState.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/ConnectionManager.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/DefaultConnectionStrategy.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/OnReconnect.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/Slice.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/SolrZkClient.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/SolrZooKeeper.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/ZkNodeProps.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/ZkStateReader.java create mode 100644 solr/src/common/org/apache/solr/common/cloud/ZooKeeperException.java create mode 100644 solr/src/java/org/apache/solr/cloud/CloudDescriptor.java create mode 100644 solr/src/java/org/apache/solr/cloud/SolrZkServer.java create mode 100644 solr/src/java/org/apache/solr/cloud/ZkController.java create mode 100644 solr/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java create mode 100644 solr/src/solrj/org/apache/solr/client/solrj/impl/CloudSolrServer.java create mode 100644 solr/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java create mode 100644 solr/src/test/org/apache/solr/cloud/AbstractZkTestCase.java create mode 100644 solr/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/BasicZkTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/CloudStateUpdateTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/ZkControllerTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/ZkNodePropsTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/ZkSolrClientTest.java create mode 100644 solr/src/test/org/apache/solr/cloud/ZkTestServer.java create mode 100644 solr/src/test/test-files/solr/solr.xml create mode 100644 solr/src/webapp/web/admin/zookeeper.jsp diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 1f94ce67028..68f6f541aab 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -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 ---------------------- diff --git a/solr/lib/log4j-over-slf4j-1.5.5.jar b/solr/lib/log4j-over-slf4j-1.5.5.jar new file mode 100644 index 00000000000..cfc2b224028 --- /dev/null +++ b/solr/lib/log4j-over-slf4j-1.5.5.jar @@ -0,0 +1,2 @@ +AnyObjectId[d32987769b4827a9c539009e804415b09b39e3ed] was removed in git history. +Apache SVN contains full history. \ No newline at end of file diff --git a/solr/lib/zookeeper-3.3.1.jar b/solr/lib/zookeeper-3.3.1.jar new file mode 100644 index 00000000000..a46d6b18418 --- /dev/null +++ b/solr/lib/zookeeper-3.3.1.jar @@ -0,0 +1,2 @@ +AnyObjectId[3b4cca851d07d2aba1630a46b283ab63f063d7c1] was removed in git history. +Apache SVN contains full history. \ No newline at end of file diff --git a/solr/src/common/org/apache/solr/common/cloud/CloudState.java b/solr/src/common/org/apache/solr/common/cloud/CloudState.java new file mode 100644 index 00000000000..c3fecc5aa3f --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/CloudState.java @@ -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> collectionStates; + private final Set liveNodes; + + public CloudState(Set liveNodes, Map> collectionStates) { + this.liveNodes = liveNodes; + this.collectionStates = collectionStates; + } + + public Map getSlices(String collection) { + Map collectionState = collectionStates.get(collection); + if(collectionState == null) { + return null; + } + return Collections.unmodifiableMap(collectionState); + } + + public Set getCollections() { + return Collections.unmodifiableSet(collectionStates.keySet()); + } + + public Map> getCollectionStates() { + return Collections.unmodifiableMap(collectionStates); + } + + public Set 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> collectionStates; + if (!onlyLiveNodes) { + List collections = zkClient.getChildren( + ZkStateReader.COLLECTIONS_ZKNODE, null); + + collectionStates = new HashMap>(); + for (String collection : collections) { + String shardIdPaths = ZkStateReader.COLLECTIONS_ZKNODE + "/" + + collection + ZkStateReader.SHARDS_ZKNODE; + List shardIdNames; + try { + shardIdNames = zkClient.getChildren(shardIdPaths, null); + } catch (KeeperException.NoNodeException e) { + // node is not valid currently + continue; + } + Map slices = new HashMap(); + for (String shardIdZkPath : shardIdNames) { + Map 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 readShards(SolrZkClient zkClient, String shardsZkPath) + throws KeeperException, InterruptedException, IOException { + + Map shardNameToProps = new HashMap(); + + if (zkClient.exists(shardsZkPath, null) == null) { + throw new IllegalStateException("Cannot find zk shards node that should exist:" + + shardsZkPath); + } + + List 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 getLiveNodes(SolrZkClient zkClient) throws KeeperException, InterruptedException { + List liveNodes = zkClient.getChildren(ZkStateReader.LIVE_NODES_ZKNODE, null); + Set liveNodesSet = new HashSet(liveNodes.size()); + liveNodesSet.addAll(liveNodes); + + return liveNodesSet; + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/ConnectionManager.java b/solr/src/common/org/apache/solr/common/cloud/ConnectionManager.java new file mode 100644 index 00000000000..367306bc904 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/ConnectionManager.java @@ -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"); + } + } +} diff --git a/solr/src/common/org/apache/solr/common/cloud/DefaultConnectionStrategy.java b/solr/src/common/org/apache/solr/common/cloud/DefaultConnectionStrategy.java new file mode 100644 index 00000000000..012a0b290d0 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/DefaultConnectionStrategy.java @@ -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); + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/OnReconnect.java b/solr/src/common/org/apache/solr/common/cloud/OnReconnect.java new file mode 100644 index 00000000000..caf749e798b --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/OnReconnect.java @@ -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(); +} diff --git a/solr/src/common/org/apache/solr/common/cloud/Slice.java b/solr/src/common/org/apache/solr/common/cloud/Slice.java new file mode 100644 index 00000000000..ccfa9b13139 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/Slice.java @@ -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 shards; + private final String name; + + public Slice(String name, Map shards) { + this.shards = shards; + this.name = name; + } + + public Map getShards() { + return Collections.unmodifiableMap(shards); + } + + public String getName() { + return name; + } +} diff --git a/solr/src/common/org/apache/solr/common/cloud/SolrZkClient.java b/solr/src/common/org/apache/solr/common/cloud/SolrZkClient.java new file mode 100644 index 00000000000..70ff66b7585 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/SolrZkClient.java @@ -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. + *

+ * 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, + CreateMode createMode) throws KeeperException, InterruptedException { + return keeper.create(path, data, acl, createMode); + } + + /** + * @param path + * @param watcher + * @return + * @throws KeeperException + * @throws InterruptedException + */ + public List 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 path=/solr/group/node 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 path=/solr/group/node 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 path=/solr/group/node 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 path=/solr/group/node 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 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; + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/SolrZooKeeper.java b/solr/src/common/org/apache/solr/common/cloud/SolrZooKeeper.java new file mode 100644 index 00000000000..8390bc0a1f6 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/SolrZooKeeper.java @@ -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; + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java b/solr/src/common/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java new file mode 100644 index 00000000000..0b425293063 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java @@ -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; + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/ZkNodeProps.java b/solr/src/common/org/apache/solr/common/cloud/ZkNodeProps.java new file mode 100644 index 00000000000..1819ee49440 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/ZkNodeProps.java @@ -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 { + + 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> entries = entrySet(); + for(Entry entry : entries) { + sb.append(entry.getKey() + "=" + entry.getValue() + "\n"); + } + return sb.toString().getBytes("UTF-8"); + } + + public String toString() { + StringBuilder sb = new StringBuilder(); + Set> entries = entrySet(); + for(Entry entry : entries) { + sb.append(entry.getKey() + "=" + entry.getValue() + "\n"); + } + return sb.toString(); + } + +} diff --git a/solr/src/common/org/apache/solr/common/cloud/ZkStateReader.java b/solr/src/common/org/apache/solr/common/cloud/ZkStateReader.java new file mode 100644 index 00000000000..2b8a138c3d7 --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/ZkStateReader.java @@ -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(0), new HashMap>(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 knownCollections = cloudState.getCollections(); + List 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 shardIds = zkClient.getChildren(COLLECTIONS_ZKNODE + "/" + + collection + SHARDS_ZKNODE, null); + CloudState cloudState = getCloudState(); + Set knownShardIds; + Map slices = cloudState.getSlices(collection); + if (slices != null) { + knownShardIds = slices.keySet(); + } else { + knownShardIds = new HashSet(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 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); + } + + }}); + + } +} diff --git a/solr/src/common/org/apache/solr/common/cloud/ZooKeeperException.java b/solr/src/common/org/apache/solr/common/cloud/ZooKeeperException.java new file mode 100644 index 00000000000..9f429f28a0c --- /dev/null +++ b/solr/src/common/org/apache/solr/common/cloud/ZooKeeperException.java @@ -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); + } + +} diff --git a/solr/src/common/org/apache/solr/common/params/CoreAdminParams.java b/solr/src/common/org/apache/solr/common/params/CoreAdminParams.java index 6a6e7268a9c..6171262df07 100644 --- a/solr/src/common/org/apache/solr/common/params/CoreAdminParams.java +++ b/solr/src/common/org/apache/solr/common/params/CoreAdminParams.java @@ -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, diff --git a/solr/src/java/org/apache/solr/cloud/CloudDescriptor.java b/solr/src/java/org/apache/solr/cloud/CloudDescriptor.java new file mode 100644 index 00000000000..df583d22c5b --- /dev/null +++ b/solr/src/java/org/apache/solr/cloud/CloudDescriptor.java @@ -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; + } +} diff --git a/solr/src/java/org/apache/solr/cloud/SolrZkServer.java b/solr/src/java/org/apache/solr/cloud/SolrZkServer.java new file mode 100644 index 00000000000..d3ff0969f5e --- /dev/null +++ b/solr/src/java/org/apache/solr/cloud/SolrZkServer.java @@ -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 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 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: + 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 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"); + } + } + } + + +} diff --git a/solr/src/java/org/apache/solr/cloud/ZkController.java b/solr/src/java/org/apache/solr/cloud/ZkController.java new file mode 100644 index 00000000000..ab4d26b9bd3 --- /dev/null +++ b/solr/src/java/org/apache/solr/cloud/ZkController.java @@ -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 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; + } + +} diff --git a/solr/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java new file mode 100644 index 00000000000..32a2ce0df51 --- /dev/null +++ b/solr/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java @@ -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; + } + + /** + *

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

+ */ + 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 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]); + } + +} diff --git a/solr/src/java/org/apache/solr/core/CoreContainer.java b/solr/src/java/org/apache/solr/core/CoreContainer.java index ef7212384f8..d6b192d9203 100644 --- a/solr/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/src/java/org/apache/solr/core/CoreContainer.java @@ -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(); } 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 names = new HashSet(); - for (int i=0; i 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 ="\n" + "\n" + " \n" + diff --git a/solr/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/src/java/org/apache/solr/core/CoreDescriptor.java index 5ad0fc9a669..1317db4fa14 100644 --- a/solr/src/java/org/apache/solr/core/CoreDescriptor.java +++ b/solr/src/java/org/apache/solr/core/CoreDescriptor.java @@ -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; + } } diff --git a/solr/src/java/org/apache/solr/core/SolrCore.java b/solr/src/java/org/apache/solr/core/SolrCore.java index 2cb1b44bcb8..72ae0b78505 100644 --- a/solr/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/src/java/org/apache/solr/core/SolrCore.java @@ -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 hide = new HashSet(); - 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 . Please "+ "update your config to use ."); } - - 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; diff --git a/solr/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/src/java/org/apache/solr/core/SolrResourceLoader.java index 7b9b646fd31..39d6a954814 100644 --- a/solr/src/java/org/apache/solr/core/SolrResourceLoader.java +++ b/solr/src/java/org/apache/solr/core/SolrResourceLoader.java @@ -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/"; diff --git a/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java index bc7ec92125f..f60669c9281 100644 --- a/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java +++ b/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java @@ -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); diff --git a/solr/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/src/java/org/apache/solr/handler/component/QueryComponent.java index c32eb1bafd7..dba1b61ee68 100644 --- a/solr/src/java/org/apache/solr/handler/component/QueryComponent.java +++ b/solr/src/java/org/apache/solr/handler/component/QueryComponent.java @@ -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 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 slices = null; + CoreDescriptor coreDescriptor = req.getCore().getCoreDescriptor(); + CloudDescriptor cloudDescriptor = coreDescriptor.getCloudDescriptor(); + ZkController zkController = coreDescriptor.getCoreContainer().getZkController(); + + + if (shards != null) { + List 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; ishards + 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 sliceShards = slice.getShards(); + + // For now, recreate the | delimited list of equivalent servers + Set 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) { diff --git a/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java index abc76275607..d813f97f196 100644 --- a/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java +++ b/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java @@ -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 searchHolder = null; try { diff --git a/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java b/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java index 0f54504395b..49eaf3383cd 100644 --- a/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java +++ b/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java @@ -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 outgoing; // requests to be sent diff --git a/solr/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/src/java/org/apache/solr/handler/component/SearchHandler.java index eb5103cf9e6..96744dc0650 100644 --- a/solr/src/java/org/apache/solr/handler/component/SearchHandler.java +++ b/solr/src/java/org/apache/solr/handler/component/SearchHandler.java @@ -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 completionService = new ExecutorCompletionService(commExecutor); Set> pending = new HashSet>(); + // 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> shardToURLs = new HashMap>(); + HttpCommComponent() { } @@ -404,7 +445,36 @@ class HttpCommComponent { } } + + // Not thread safe... don't use in Callable. + // Don't modify the returned URL list. + private List getURLs(String shard) { + List urls = shardToURLs.get(shard); + if (urls==null) { + urls = StrUtils.splitSmart(shard,"|",true); + + // convert shard to URL + for (int i=0; i 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 urls = getURLs(shard); + Callable task = new Callable() { 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) { diff --git a/solr/src/java/org/apache/solr/handler/component/ShardRequest.java b/solr/src/java/org/apache/solr/handler/component/ShardRequest.java index 9a11d313637..d8828cb24ac 100755 --- a/solr/src/java/org/apache/solr/handler/component/ShardRequest.java +++ b/solr/src/java/org/apache/solr/handler/component/ShardRequest.java @@ -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; diff --git a/solr/src/java/org/apache/solr/handler/component/ShardResponse.java b/solr/src/java/org/apache/solr/handler/component/ShardResponse.java index 7d07b82f017..87f89571c44 100755 --- a/solr/src/java/org/apache/solr/handler/component/ShardResponse.java +++ b/solr/src/java/org/apache/solr/handler/component/ShardResponse.java @@ -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; } } diff --git a/solr/src/java/org/apache/solr/handler/component/TermsComponent.java b/solr/src/java/org/apache/solr/handler/component/TermsComponent.java index 1ac33c13acd..091917292b0 100644 --- a/solr/src/java/org/apache/solr/handler/component/TermsComponent.java +++ b/solr/src/java/org/apache/solr/handler/component/TermsComponent.java @@ -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; } diff --git a/solr/src/solrj/org/apache/solr/client/solrj/impl/CloudSolrServer.java b/solr/src/solrj/org/apache/solr/client/solrj/impl/CloudSolrServer.java new file mode 100644 index 00000000000..60955330996 --- /dev/null +++ b/solr/src/solrj/org/apache/solr/client/solrj/impl/CloudSolrServer.java @@ -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 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 slices = cloudState.getSlices(collection); + Set 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 nodes = new HashMap(); + List urlList = new ArrayList(); + 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; + } + } + } +} diff --git a/solr/src/solrj/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java b/solr/src/solrj/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java index 4a02b4cd526..cf55dd3b256 100644 --- a/solr/src/solrj/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java +++ b/solr/src/solrj/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java @@ -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 NOT be used for * indexing. Also see the wiki page. *

* It offers automatic failover when a server goes down and it detects when the server comes back up. *

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

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

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

* When to use this?
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 Load balancing on Wikipedia * * @since solr 1.4 */ public class LBHttpSolrServer extends SolrServer { - private final CopyOnWriteArrayList aliveServers = new CopyOnWriteArrayList(); - private final CopyOnWriteArrayList zombieServers = new CopyOnWriteArrayList(); + + + // keys to the maps are currently of the form "http://localhost:8983/solr" + // which should be equivalent to CommonsHttpSolrServer.getBaseURL() + private final Map aliveServers = new LinkedHashMap(); + // access to aliveServers should be synchronized on itself + + private final Map zombieServers = new ConcurrentHashMap(); + + // 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 servers; + protected int numDeadServersToTry; + + public Req(SolrRequest request, List servers) { + this.request = request; + this.servers = servers; + this.numDeadServersToTry = servers.size(); + } + + public SolrRequest getRequest() { + return request; + } + public List 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 rsp; + + /** The response from the server */ + public NamedList 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 skipped = new ArrayList(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 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 justFailed = null; + + for (int attempts=0; attempts(); + 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 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) { + private static Runnable getAliveCheckRunner(final WeakReference 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 } diff --git a/solr/src/test/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/src/test/org/apache/solr/BaseDistributedSearchTestCase.java index a1864c03721..92ea4c9f631 100644 --- a/solr/src/test/org/apache/solr/BaseDistributedSearchTestCase.java +++ b/solr/src/test/org/apache/solr/BaseDistributedSearchTestCase.java @@ -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 jettys = new ArrayList(); 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 replicas = new ArrayList(Arrays.asList(deadServers)); + Collections.shuffle(replicas, r); + replicas.add(r.nextInt(nDeadServers+1), shard); + for (int i=0; i 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(); + } +} diff --git a/solr/src/test/org/apache/solr/cloud/AbstractZkTestCase.java b/solr/src/test/org/apache/solr/cloud/AbstractZkTestCase.java new file mode 100644 index 00000000000..8da1329b3a7 --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/AbstractZkTestCase.java @@ -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(); + } +} diff --git a/solr/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java new file mode 100644 index 00000000000..9e6beb4425d --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java @@ -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 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 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 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(); + } +} diff --git a/solr/src/test/org/apache/solr/cloud/ZkControllerTest.java b/solr/src/test/org/apache/solr/cloud/ZkControllerTest.java new file mode 100644 index 00000000000..ba809e83217 --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/ZkControllerTest.java @@ -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 slices = cloudInfo.getSlices("collection1"); + assertNotNull(slices); + + for (Slice slice : slices.values()) { + Map 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(); + } +} diff --git a/solr/src/test/org/apache/solr/cloud/ZkNodePropsTest.java b/solr/src/test/org/apache/solr/cloud/ZkNodePropsTest.java new file mode 100644 index 00000000000..b5339742a10 --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/ZkNodePropsTest.java @@ -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")); + } +} diff --git a/solr/src/test/org/apache/solr/cloud/ZkSolrClientTest.java b/solr/src/test/org/apache/solr/cloud/ZkSolrClientTest.java new file mode 100644 index 00000000000..c8a0bce9b1b --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/ZkSolrClientTest.java @@ -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(); + } + +} diff --git a/solr/src/test/org/apache/solr/cloud/ZkTestServer.java b/solr/src/test/org/apache/solr/cloud/ZkTestServer.java new file mode 100644 index 00000000000..ef1275823cb --- /dev/null +++ b/solr/src/test/org/apache/solr/cloud/ZkTestServer.java @@ -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 parseHostPortList(String hplist) { + ArrayList alist = new ArrayList(); + 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; + } +} diff --git a/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java b/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java index 6be552f8f5c..508efbcd60b 100644 --- a/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java +++ b/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java @@ -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"); diff --git a/solr/src/test/test-files/solr/solr.xml b/solr/src/test/test-files/solr/solr.xml new file mode 100644 index 00000000000..f92f6e46fe0 --- /dev/null +++ b/solr/src/test/test-files/solr/solr.xml @@ -0,0 +1,34 @@ + + + + + + + + + + + diff --git a/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java b/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java index 5078514b846..8910806c6bd 100644 --- a/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java +++ b/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java @@ -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 + "]"; } } diff --git a/solr/src/webapp/web/admin/index.jsp b/solr/src/webapp/web/admin/index.jsp index b2ee5449329..1cfe96659a7 100644 --- a/solr/src/webapp/web/admin/index.jsp +++ b/solr/src/webapp/web/admin/index.jsp @@ -50,6 +50,7 @@ [Statistics] [Info] [Distribution] + [ZooKeeper] [Ping] [Logging] diff --git a/solr/src/webapp/web/admin/zookeeper.jsp b/solr/src/webapp/web/admin/zookeeper.jsp new file mode 100644 index 00000000000..3c8492d4013 --- /dev/null +++ b/solr/src/webapp/web/admin/zookeeper.jsp @@ -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" %> + +
+

Zookeeper Browser

+ +<% + 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"; +%> + +
+ + + + + + + +
+ <% + XML.escapeCharData(printer.zkClient == null ? "Disconnected" + : ("Connected to zookeeper " + printer.keeperAddr), out); + %> + + Connect to different zookeeper: + + + +
+
+ + +<% + try { + printer.print(path); + } finally { + printer.close(); + } +%> + + + + +<%!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(""); + out.print(""); + + if (detail) { + out.print(""); + } + + out.print(""); + + out.print("
"); + 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("
"); + printZnode(path); + out.print("
"); + printTree(path); + out.print("
"); + } + + 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("
"); + levelchange = false; + + for (int i=0; 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(""); + xmlescape(label); + out.print(""); + + } 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 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("

"); + xmlescape(path); + out.print("

"); + + 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("
"); + out.println("
"); + } + + down(); + + } catch (KeeperException e) { + exception(e); + return; + } catch (InterruptedException e) { + exception(e); + } + } + }%> \ No newline at end of file