You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/02/06 20:17:58 UTC

svn commit: r1443145 - in /accumulo/trunk: fate/src/main/java/org/apache/accumulo/fate/zookeeper/ proxy/ test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ test/system/randomwalk/conf/modules/

Author: ecn
Date: Wed Feb  6 19:17:58 2013
New Revision: 1443145

URL: http://svn.apache.org/viewvc?rev=1443145&view=rev
Log:
ACCUMULO-636 randomly shutdown every now and then; license headers

Added:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java   (with props)
Modified:
    accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
    accumulo/trunk/proxy/proxy.properties
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
    accumulo/trunk/test/system/randomwalk/conf/modules/Concurrent.xml

Modified: accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java?rev=1443145&r1=1443144&r2=1443145&view=diff
==============================================================================
--- accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java (original)
+++ accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java Wed Feb  6 19:17:58 2013
@@ -223,7 +223,7 @@ public class ZooLock implements Watcher 
             } else if (asyncLock != null && event.getType() == EventType.NodeDeleted && event.getPath().equals(path + "/" + asyncLock)) {
               failedToAcquireLock();
             } else if(event.getState() != KeeperState.Expired) {
-              log.warn("Unexpected event wathcing lock node "+event+" "+asyncLockPath);
+              log.warn("Unexpected event watching lock node "+event+" "+asyncLockPath);
               try {
                 Stat stat2 = zooKeeper.getStatus(asyncLockPath, this);
                 if(stat2 == null){

Modified: accumulo/trunk/proxy/proxy.properties
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/proxy.properties?rev=1443145&r1=1443144&r2=1443145&view=diff
==============================================================================
--- accumulo/trunk/proxy/proxy.properties (original)
+++ accumulo/trunk/proxy/proxy.properties Wed Feb  6 19:17:58 2013
@@ -1,5 +1,5 @@
 org.apache.accumulo.proxy.ProxyServer.useMockInstance=false
-org.apache.accumulo.proxy.ProxyServer.useMiniAccumulo=false
+org.apache.accumulo.proxy.ProxyServer.useMiniAccumulo=true
 org.apache.accumulo.proxy.ProxyServer.protocolFactory=org.apache.thrift.protocol.TCompactProtocol$Factory
 org.apache.accumulo.proxy.ProxyServer.port=42424
 

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java?rev=1443145&r1=1443144&r2=1443145&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java Wed Feb  6 19:17:58 2013
@@ -1,3 +1,19 @@
+/*
+ * 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.accumulo.test.randomwalk.concurrent;
 
 import java.util.Properties;

Added: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java?rev=1443145&view=auto
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java (added)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java Wed Feb  6 19:17:58 2013
@@ -0,0 +1,57 @@
+/*
+ * 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.accumulo.test.randomwalk.concurrent;
+
+import java.util.Properties;
+
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.master.state.SetGoalState;
+import org.apache.accumulo.server.security.SecurityConstants;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+import org.apache.accumulo.trace.instrument.Tracer;
+
+public class Shutdown extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception  {
+    log.debug("shutting down");
+    SetGoalState.main(new String[]{MasterGoalState.CLEAN_STOP.name()});
+    
+    while (!state.getConnector().instanceOperations().getTabletServers().isEmpty()) {
+      UtilWaitThread.sleep(1000);
+    }
+    
+    while (true) {
+        try {
+          Client client = MasterClient.getConnection(HdfsZooInstance.getInstance());
+          client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
+        } catch (Exception e) {
+          // assume this is due to server shutdown
+          break;
+        }
+        UtilWaitThread.sleep(1000);
+    }
+
+    log.debug("tablet servers stopped");
+  }
+  
+}

Propchange: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java?rev=1443145&r1=1443144&r2=1443145&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java Wed Feb  6 19:17:58 2013
@@ -18,16 +18,36 @@ package org.apache.accumulo.test.randomw
 
 import java.util.Properties;
 
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.master.state.SetGoalState;
+import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
+import org.apache.accumulo.trace.instrument.Tracer;
 
 public class StartAll extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
     log.info("Starting all servers");
+    SetGoalState.main(new String[]{MasterGoalState.NORMAL.name()});
     Process exec = Runtime.getRuntime().exec(new String[]{System.getenv().get("ACCUMULO_HOME") + "/bin/start-all.sh"});
     exec.waitFor();
+    while (true) {
+      try {
+        Client client = MasterClient.getConnection(HdfsZooInstance.getInstance());
+        MasterMonitorInfo masterStats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
+        if (!masterStats.tServerInfo.isEmpty())
+          break;
+      } catch (Exception ex) {
+        UtilWaitThread.sleep(1000);
+      }
+    }
   }
   
 }

Modified: accumulo/trunk/test/system/randomwalk/conf/modules/Concurrent.xml
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/system/randomwalk/conf/modules/Concurrent.xml?rev=1443145&r1=1443144&r2=1443145&view=diff
==============================================================================
--- accumulo/trunk/test/system/randomwalk/conf/modules/Concurrent.xml (original)
+++ accumulo/trunk/test/system/randomwalk/conf/modules/Concurrent.xml Wed Feb  6 19:17:58 2013
@@ -46,6 +46,7 @@
   <edge id="ct.CheckBalance" weight="1000"/>
   <edge id="ct.StopTabletServer" weight="1000"/>
   <edge id="ct.StartAll" weight="1000"/>
+  <edge id="ct.Shutdown" weight="100"/>
   <edge id="ct.Config" weight="1000"/>
   <edge id="END" weight="1"/>
 </node>
@@ -147,4 +148,8 @@
   <edge id="dummy.ToAll" weight="1"/>
 </node>
 
+<node id="ct.Shutdown">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
 </module>