You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by ma...@apache.org on 2009/12/14 22:26:21 UTC
svn commit: r890484 - in
/lucene/solr/branches/cloud/src/java/org/apache/solr:
core/ZooKeeperController.java util/ZooPut.java util/zookeeper/
util/zookeeper/CountdownWatcher.java
Author: markrmiller
Date: Mon Dec 14 21:26:19 2009
New Revision: 890484
URL: http://svn.apache.org/viewvc?rev=890484&view=rev
Log:
introduce CountdownWatcher by Patrick Hunt for client sync client connect
Added:
lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/
lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/CountdownWatcher.java
Modified:
lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java
lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java
Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java?rev=890484&r1=890483&r2=890484&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java Mon Dec 14 21:26:19 2009
@@ -6,17 +6,16 @@
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import javax.xml.parsers.ParserConfigurationException;
import org.apache.solr.common.SolrException;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.util.ZooPut;
+import org.apache.solr.util.zookeeper.CountdownWatcher;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
@@ -24,7 +23,7 @@
/**
* Handle ZooKeeper interactions.
*/
-public class ZooKeeperController implements Watcher {
+public class ZooKeeperController {
private static final String CONFIGS_NODE = "configs";
private static Logger log = LoggerFactory
@@ -35,8 +34,6 @@
private String configName;
private String collectionName;
-
- private boolean connected = false;
/**
* @param zookeeperHost ZooKeeper host service
@@ -46,28 +43,22 @@
this.collectionName = collection;
+ CountdownWatcher countdownWatcher = new CountdownWatcher("ZooKeeperController");
try {
- keeper = new ZooKeeper(zookeeperHost, 10000, this);
+ keeper = new ZooKeeper(zookeeperHost, 10000, countdownWatcher);
- // TODO: nocommit: this is asynchronous - think about how to deal with connection
- // lost, and other failures
- synchronized (this) {
- while (!connected) {
- try {
- this.wait();
- } catch (InterruptedException e) {
- // nocommit
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
+ countdownWatcher.waitForConnected(5000);
loadConfigPath();
register();
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Can't create ZooKeeper instance", e);
+ } catch (InterruptedException e) {
+ // nocommit
+ } catch (TimeoutException e) {
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+ "Can't create ZooKeeper instance", e);
}
}
@@ -89,24 +80,6 @@
}
}
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatchedEvent)
- */
- public void process(WatchedEvent event) {
- // nocommit
- System.out.println("ZooKeeper Event:" + event);
- // nocommit: consider how we want to accomplish this
- if (event.getState() == KeeperState.SyncConnected) {
- synchronized (this) {
- connected = true;
- this.notify();
- }
- }
- }
-
private void loadConfigPath() {
// nocommit: load all config at once or organize differently
try {
Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java?rev=890484&r1=890483&r2=890484&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java Mon Dec 14 21:26:19 2009
@@ -3,42 +3,28 @@
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+import org.apache.solr.util.zookeeper.CountdownWatcher;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
/**
* Util for uploading and updating files in ZooKeeper.
*
*/
-public class ZooPut implements Watcher {
+public class ZooPut {
private ZooKeeper keeper;
private boolean closeKeeper = true;
-
- private boolean connected = false;
- public ZooPut(String host) throws IOException {
- keeper = new ZooKeeper(host, 10000, this);
- // TODO: nocommit: this is asynchronous - think about how to deal with connection
- // lost, and other failures
- synchronized (this) {
- while (!connected) {
- try {
- this.wait();
- } catch (InterruptedException e) {
- // nocommit
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
+ public ZooPut(String host) throws IOException, InterruptedException, TimeoutException {
+ CountdownWatcher countdownWatcher = new CountdownWatcher("ZooPut:" + this);
+ keeper = new ZooKeeper(host, 10000, countdownWatcher);
+ countdownWatcher.waitForConnected(5000);
}
public ZooPut(ZooKeeper keeper) throws IOException {
@@ -151,15 +137,4 @@
zooPut.close();
}
- public void process(WatchedEvent event) {
- // nocommit: consider how we want to accomplish this
- if (event.getState() == KeeperState.SyncConnected) {
- synchronized (this) {
- connected = true;
- this.notify();
- }
- }
-
- }
-
}
Added: lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/CountdownWatcher.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/CountdownWatcher.java?rev=890484&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/CountdownWatcher.java (added)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/util/zookeeper/CountdownWatcher.java Mon Dec 14 21:26:19 2009
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util.zookeeper;
+
+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;
+
+public class CountdownWatcher implements Watcher {
+ protected static final Logger LOG = LoggerFactory
+ .getLogger(CountdownWatcher.class);
+
+ private final String name;
+
+ private CountDownLatch clientConnected;
+
+ private KeeperState state;
+
+ private boolean connected;
+
+ public CountdownWatcher(String name) {
+ this.name = name;
+ reset();
+ }
+
+ private synchronized void reset() {
+ clientConnected = new CountDownLatch(1);
+ state = KeeperState.Disconnected;
+ connected = false;
+ }
+
+ public synchronized void process(WatchedEvent event) {
+ LOG.info("Watcher " + name + " got event " + event);
+
+ state = event.getState();
+ if (state == KeeperState.SyncConnected) {
+ connected = true;
+ clientConnected.countDown();
+ } else {
+ connected = false;
+ }
+ notifyAll();
+ }
+
+ public synchronized boolean isConnected() {
+ return connected;
+ }
+
+ public synchronized KeeperState state() {
+ return state;
+ }
+
+ public synchronized void waitForConnected(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 connect");
+
+ }
+ }
+
+ 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");
+
+ }
+ }
+}