You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2014/08/29 14:30:54 UTC

svn commit: r1621294 [2/2] - in /lucene/dev/trunk/solr: ./ cloud-dev/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/test/org/apache/solr/cloud/ example/solr/ solrj/src/java/org/apache/solr/common/ solrj/src/java/org/...

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java?rev=1621294&r1=1621293&r2=1621294&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java Fri Aug 29 12:30:53 2014
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkCredentialsProvider.ZkCredentials;
 import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,12 +34,19 @@ import org.slf4j.LoggerFactory;
 public abstract class ZkClientConnectionStrategy {
   private static Logger log = LoggerFactory.getLogger(ZkClientConnectionStrategy.class);
   
+  private volatile ZkCredentialsProvider zkCredentialsToAddAutomatically;
+  private volatile boolean zkCredentialsToAddAutomaticallyUsed;
+  
   private List<DisconnectedListener> disconnectedListeners = new ArrayList<>();
   private List<ConnectedListener> connectedListeners = new ArrayList<>();
   
   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 ZkClientConnectionStrategy() {
+    zkCredentialsToAddAutomaticallyUsed = false;
+  }
+  
   public synchronized void disconnected() {
     for (DisconnectedListener listener : disconnectedListeners) {
       try {
@@ -80,4 +88,26 @@ public abstract class ZkClientConnection
     public abstract void update(SolrZooKeeper zooKeeper) throws InterruptedException, TimeoutException, IOException;
   }
   
+  public void setZkCredentialsToAddAutomatically(ZkCredentialsProvider zkCredentialsToAddAutomatically) {
+    if (zkCredentialsToAddAutomaticallyUsed || (zkCredentialsToAddAutomatically == null)) 
+      throw new RuntimeException("Cannot change zkCredentialsToAddAutomatically after it has been (connect or reconnect was called) used or to null");
+    this.zkCredentialsToAddAutomatically = zkCredentialsToAddAutomatically;
+  }
+  
+  public boolean hasZkCredentialsToAddAutomatically() {
+    return zkCredentialsToAddAutomatically != null;
+  }
+  
+  protected SolrZooKeeper createSolrZooKeeper(final String serverAddress, final int zkClientTimeout,
+      final Watcher watcher) throws IOException {
+    SolrZooKeeper result = new SolrZooKeeper(serverAddress, zkClientTimeout, watcher);
+    
+    zkCredentialsToAddAutomaticallyUsed = true;
+    for (ZkCredentials zkCredentials : zkCredentialsToAddAutomatically.getCredentials()) {
+      result.addAuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth());
+    }
+
+    return result;
+  }
+  
 }

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java?rev=1621294&r1=1621293&r2=1621294&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java Fri Aug 29 12:30:53 2014
@@ -17,19 +17,14 @@ package org.apache.solr.common.cloud;
  * limitations under the License.
  */
 
-import java.util.List;
-
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
 
 
 public class ZkCmdExecutor {
   private long retryDelay = 1500L; // 1 second would match timeout, so 500 ms over for padding
   private int retryCount;
-  private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
   private double timeouts;
   
   /**
@@ -45,14 +40,6 @@ public class ZkCmdExecutor {
     this.retryCount = Math.round(0.5f * ((float)Math.sqrt(8.0f * timeouts + 1.0f) - 1.0f)) + 1;
   }
   
-  public List<ACL> getAcl() {
-    return acl;
-  }
-  
-  public void setAcl(List<ACL> acl) {
-    this.acl = acl;
-  }
-  
   public long getRetryDelay() {
     return retryDelay;
   }

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java?rev=1621294&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCredentialsProvider.java Fri Aug 29 12:30:53 2014
@@ -0,0 +1,45 @@
+package org.apache.solr.common.cloud;
+
+import java.util.Collection;
+
+/*
+ * 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 ZkCredentialsProvider {
+  
+  public class ZkCredentials {
+    String scheme;
+    byte[] auth;
+    
+    public ZkCredentials(String scheme, byte[] auth) {
+      super();
+      this.scheme = scheme;
+      this.auth = auth;
+    }
+    
+    String getScheme() {
+      return scheme;
+    }
+    
+    byte[] getAuth() {
+      return auth;
+    }
+  }
+  
+  Collection<ZkCredentials> getCredentials();
+
+}

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java?rev=1621294&r1=1621293&r2=1621294&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java Fri Aug 29 12:30:53 2014
@@ -29,13 +29,10 @@ import org.apache.commons.io.IOUtils;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.io.Files;
-
 public class MiniSolrCloudCluster {
   
   private static Logger log = LoggerFactory.getLogger(MiniSolrCloudCluster.class);
@@ -70,8 +67,7 @@ public class MiniSolrCloudCluster {
         AbstractZkTestCase.TIMEOUT, 45000, null);
       zkClient.makePath("/solr", false, true);
       is = new FileInputStream(solrXml);
-      zkClient.create("/solr/solr.xml", IOUtils.toByteArray(is),
-        ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, true);
+      zkClient.create("/solr/solr.xml", IOUtils.toByteArray(is), CreateMode.PERSISTENT, true);
     } finally {
       IOUtils.closeQuietly(is);
       if (zkClient != null) zkClient.close();