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 2013/12/01 22:58:45 UTC

svn commit: r1546863 - in /lucene/dev/trunk/solr: solrj/src/java/org/apache/solr/common/cloud/ test-framework/src/java/org/apache/solr/ test-framework/src/java/org/apache/solr/cloud/

Author: markrmiller
Date: Sun Dec  1 21:58:45 2013
New Revision: 1546863

URL: http://svn.apache.org/r1546863
Log:
SOLR-5482: add option to simulate hard jetty fails with iptables

Added:
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/IpTables.java   (with props)
Modified:
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZooKeeper.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZooKeeper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZooKeeper.java?rev=1546863&r1=1546862&r2=1546863&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZooKeeper.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZooKeeper.java Sun Dec  1 21:58:45 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.net.SocketAddress;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
 
@@ -45,6 +46,10 @@ public class SolrZooKeeper extends ZooKe
     return cnxn;
   }
   
+  public SocketAddress getSocketAddress() {
+    return testableLocalSocketAddress();
+  }
+  
   /**
    * Cause this ZooKeeper object to stop receiving from the ZooKeeperServer
    * for the given number of milliseconds.

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1546863&r1=1546862&r2=1546863&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Sun Dec  1 21:58:45 2013
@@ -47,6 +47,7 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.QuickPatchThreadsFilter;
 import org.apache.lucene.util._TestUtil;
 import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.cloud.IpTables;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
@@ -143,6 +144,8 @@ public abstract class SolrTestCaseJ4 ext
     System.clearProperty("tests.shardhandler.randomSeed");
     System.clearProperty("enable.update.log");
     System.clearProperty("useCompoundFile");
+    
+    IpTables.unblockAllPorts();
   }
 
   private static boolean changedFactory = false;

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1546863&r1=1546862&r2=1546863&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Sun Dec  1 21:58:45 2013
@@ -210,7 +210,7 @@ public abstract class AbstractFullDistri
   }
   
   @AfterClass
-  public static void afterClass() {
+  public static void afterClass() throws Exception {
     System.clearProperty("solrcloud.update.delay");
     System.clearProperty("genericCoreNodeNames");
   }

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java?rev=1546863&r1=1546862&r2=1546863&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Sun Dec  1 21:58:45 2013
@@ -17,12 +17,15 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.servlet.Filter;
+
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -226,6 +229,21 @@ public class ChaosMonkey {
   }
   
   public static void kill(CloudJettyRunner cjetty) throws Exception {
+    FilterHolder filterHolder = cjetty.jetty.getDispatchFilter();
+    if (filterHolder != null) {
+      Filter filter = filterHolder.getFilter();
+      if (filter != null) {
+        CoreContainer cores = ((SolrDispatchFilter) filter).getCores();
+        if (cores != null) {
+          int zklocalport = ((InetSocketAddress) cores.getZkController()
+              .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
+          IpTables.blockPort(zklocalport);
+        }
+      }
+    }
+
+    IpTables.blockPort(cjetty.jetty.getLocalPort());
+    
     JettySolrRunner jetty = cjetty.jetty;
     monkeyLog("kill shard! " + jetty.getLocalPort());
     
@@ -534,7 +552,8 @@ public class ChaosMonkey {
   }
   
   public static boolean start(JettySolrRunner jetty) throws Exception {
-    
+
+    IpTables.unblockPort(jetty.getLocalPort());
     try {
       jetty.start();
     } catch (Exception e) {
@@ -555,6 +574,18 @@ public class ChaosMonkey {
         }
       }
     }
+    FilterHolder filterHolder = jetty.getDispatchFilter();
+    if (filterHolder != null) {
+      Filter filter = filterHolder.getFilter();
+      if (filter != null) {
+        CoreContainer cores = ((SolrDispatchFilter) filter).getCores();
+        if (cores != null) {
+          int zklocalport = ((InetSocketAddress) cores.getZkController()
+              .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
+          IpTables.unblockPort(zklocalport);
+        }
+      }
+    }
     return true;
   }
 

Added: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/IpTables.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/IpTables.java?rev=1546863&view=auto
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/IpTables.java (added)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/IpTables.java Sun Dec  1 21:58:45 2013
@@ -0,0 +1,116 @@
+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.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * To use, tests must be able to run iptables, eg sudo chmod u+s iptables
+ */
+public class IpTables {
+  static final Logger log = LoggerFactory
+      .getLogger(IpTables.class);
+  
+  private static boolean ENABLED = Boolean.getBoolean("solr.tests.use.iptables");
+  static class ThreadPumper {
+
+    public ThreadPumper() {}
+    
+    public static Thread start(final InputStream from, final OutputStream to, final boolean verbose) {
+      Thread t = new Thread() {
+        @Override
+        public void run() {
+          try {
+            byte [] buffer = new byte [1024];
+            int len;
+            while ((len = from.read(buffer)) != -1) {
+              if (verbose) {
+                to.write(buffer, 0, len);
+              }
+            }
+          } catch (IOException e) {
+            System.err.println("Couldn't pipe from the forked process: " + e.toString());
+          }
+        }
+      };
+      t.start();
+      return t;
+    }
+  }
+  
+  private static Set<Integer> BLOCK_PORTS = Collections.synchronizedSet(new HashSet<Integer>());
+  
+  public static void blockPort(int port) throws IOException,
+      InterruptedException {
+    if (ENABLED) {
+      log.info("Block port with iptables: " + port);
+      BLOCK_PORTS.add(port);
+      runCmd(("iptables -A INPUT -p tcp --dport " + port + " -j DROP")
+          .split("\\s"));
+      runCmd(("iptables -A OUTPUT -p tcp --dport " + port + " -j DROP")
+          .split("\\s"));
+    }
+  }
+  
+  public static void unblockPort(int port) throws IOException,
+      InterruptedException {
+    if (ENABLED) {
+      log.info("Unblock port with iptables: " + port);
+      runCmd(("iptables -D INPUT -p tcp --dport " + port + " -j DROP")
+          .split("\\s"));
+      runCmd(("iptables -D OUTPUT -p tcp --dport " + port + " -j DROP")
+          .split("\\s"));
+    }
+  }
+  
+  public static void unblockAllPorts() throws IOException, InterruptedException {
+    if (ENABLED) {
+      log.info("Unblocking any ports previously blocked with iptables...");
+      for (Integer port : BLOCK_PORTS) {
+        IpTables.unblockPort(port);
+      }
+    }
+  }
+  
+  private static void runCmd(String[] cmd) throws IOException, InterruptedException {
+    ProcessBuilder pb = new ProcessBuilder(cmd);
+
+    pb.redirectErrorStream(true);
+    Process p = pb.start();
+
+    // We pump everything to stderr.
+    PrintStream childOut = System.err; 
+    Thread stdoutPumper = ThreadPumper.start(p.getInputStream(), childOut, true);
+    Thread stderrPumper = ThreadPumper.start(p.getErrorStream(), childOut, true);
+    if (true) childOut.println(">>> Begin subprocess output");
+    p.waitFor();
+    stdoutPumper.join();
+    stderrPumper.join();
+    if (true) childOut.println("<<< End subprocess output");
+  }
+}