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 2010/01/02 20:36:09 UTC

svn commit: r895272 - in /lucene/solr/branches/cloud/src/java/org/apache/solr/cloud: ConnectionManager.java DefaultConnectionStrategy.java SolrZkClient.java ZkClientConnectionStrategy.java

Author: markrmiller
Date: Sat Jan  2 19:36:08 2010
New Revision: 895272

URL: http://svn.apache.org/viewvc?rev=895272&view=rev
Log:
add callback to update ZooKeeper in connection strategy

Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ConnectionManager.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/DefaultConnectionStrategy.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkClientConnectionStrategy.java

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ConnectionManager.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ConnectionManager.java?rev=895272&r1=895271&r2=895272&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ConnectionManager.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ConnectionManager.java Sat Jan  2 19:36:08 2010
@@ -1,22 +1,5 @@
 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 java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeoutException;
@@ -43,8 +26,11 @@
 
   private int zkClientTimeout;
 
-  public ConnectionManager(String name, String zkServerAddress, int zkClientTimeout, ZkClientConnectionStrategy strat) {
+  private SolrZkClient client;
+
+  public ConnectionManager(String name, SolrZkClient client, String zkServerAddress, int zkClientTimeout, ZkClientConnectionStrategy strat) {
     this.name = name;
+    this.client = client;
     this.connectionStrategy = strat;
     this.zkServerAddress = zkServerAddress;
     this.zkClientTimeout = zkClientTimeout;
@@ -74,7 +60,12 @@
       // nocommit : close old ZooKeeper client?
 
       try {
-        connectionStrategy.reconnect(zkServerAddress, zkClientTimeout, this);
+        connectionStrategy.reconnect(zkServerAddress, zkClientTimeout, this, new ZkClientConnectionStrategy.ZkUpdate() {
+          @Override
+          public void update(ZooKeeper keeper) {
+           client.updateKeeper(keeper);
+          }
+        });
       } catch (IOException e) {
         // TODO Auto-generated catch block
         e.printStackTrace();

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/DefaultConnectionStrategy.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/DefaultConnectionStrategy.java?rev=895272&r1=895271&r2=895272&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/DefaultConnectionStrategy.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/DefaultConnectionStrategy.java Sat Jan  2 19:36:08 2010
@@ -22,16 +22,19 @@
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 
+/**
+ *
+ */
 public class DefaultConnectionStrategy extends ZkClientConnectionStrategy {
 
   @Override
-  public ZooKeeper connect(String serverAddress, int timeout, Watcher watcher) throws IOException {
-    return new ZooKeeper(serverAddress, timeout, watcher);
+  public void connect(String serverAddress, int timeout, Watcher watcher, ZkUpdate updater) throws IOException {
+    updater.update(new ZooKeeper(serverAddress, timeout, watcher));
   }
 
   @Override
-  public ZooKeeper reconnect(String serverAddress, int timeout, Watcher watcher) throws IOException {
-    return new ZooKeeper(serverAddress, timeout, watcher);
+  public void reconnect(String serverAddress, int timeout, Watcher watcher, ZkUpdate updater) throws IOException {
+    updater.update(new ZooKeeper(serverAddress, timeout, watcher));
   }
 
 }

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java?rev=895272&r1=895271&r2=895272&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java Sat Jan  2 19:36:08 2010
@@ -24,6 +24,7 @@
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.solr.cloud.ZkClientConnectionStrategy.ZkUpdate;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
@@ -55,9 +56,14 @@
   public SolrZkClient(String zkServerAddress, int zkClientTimeout,
       ZkClientConnectionStrategy strat) throws InterruptedException,
       TimeoutException, IOException {
-    connManager = new ConnectionManager("ZooKeeperConnection Watcher",
+    connManager = new ConnectionManager("ZooKeeperConnection Watcher", this,
         zkServerAddress, zkClientTimeout, strat);
-    this.keeper = strat.connect(zkServerAddress, zkClientTimeout, connManager);
+    strat.connect(zkServerAddress, zkClientTimeout, connManager, new ZkUpdate() {
+      @Override
+      public void update(ZooKeeper zooKeeper) {
+        keeper = zooKeeper;
+      }
+    });
     connManager.waitForConnected(CONNECT_TIMEOUT);
   }
 
@@ -314,4 +320,8 @@
   public void close() throws InterruptedException {
     keeper.close();
   }
+
+  void updateKeeper(ZooKeeper keeper) {
+   this.keeper = keeper;
+  }
 }

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkClientConnectionStrategy.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkClientConnectionStrategy.java?rev=895272&r1=895271&r2=895272&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkClientConnectionStrategy.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkClientConnectionStrategy.java Sat Jan  2 19:36:08 2010
@@ -26,6 +26,11 @@
  *
  */
 public abstract class ZkClientConnectionStrategy {
-  public abstract ZooKeeper connect(String zkServerAddress, int zkClientTimeout, Watcher watcher) throws IOException;
-  public abstract ZooKeeper reconnect(String serverAddress, int zkClientTimeout, Watcher watcher) throws IOException;
+  public abstract void connect(String zkServerAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException;
+  public abstract void reconnect(String serverAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException;
+  
+  public static abstract class ZkUpdate {
+    public abstract void update(ZooKeeper zooKeeper);
+  }
+  
 }