accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e..@apache.org
Subject svn commit: r1482989 - /accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
Date Wed, 15 May 2013 18:27:13 GMT
Author: ecn
Date: Wed May 15 18:27:12 2013
New Revision: 1482989

URL: http://svn.apache.org/r1482989
Log:
ACCUMULO-1418 use ZooReaderWriter, and not raw ZooKeeper

Modified:
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java?rev=1482989&r1=1482988&r2=1482989&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
(original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
Wed May 15 18:27:12 2013
@@ -23,15 +23,13 @@ import java.util.Stack;
 import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 
-import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
 import org.xml.sax.Attributes;
 import org.xml.sax.SAXException;
 import org.xml.sax.helpers.DefaultHandler;
@@ -39,11 +37,11 @@ import org.xml.sax.helpers.DefaultHandle
 public class RestoreZookeeper {
   
   private static class Restore extends DefaultHandler {
-    ZooKeeper zk = null;
+    IZooReaderWriter zk = null;
     Stack<String> cwd = new Stack<String>();
     boolean overwrite = false;
     
-    Restore(ZooKeeper zk, boolean overwrite) {
+    Restore(IZooReaderWriter zk, boolean overwrite) {
       this.zk = zk;
       this.overwrite = overwrite;
     }
@@ -79,7 +77,7 @@ public class RestoreZookeeper {
         data = Base64.decodeBase64(value.getBytes());
       try {
         try {
-          ZooUtil.putPersistentData(zk, path, data, overwrite ? NodeExistsPolicy.OVERWRITE
: NodeExistsPolicy.FAIL);
+          zk.putPersistentData(path, data, overwrite ? NodeExistsPolicy.OVERWRITE : NodeExistsPolicy.FAIL);
         } catch (KeeperException e) {
           if (e.code().equals(KeeperException.Code.NODEEXISTS))
             throw new RuntimeException(path + " exists.  Remove it first.");
@@ -98,8 +96,6 @@ public class RestoreZookeeper {
   public static void main(String[] args) throws Exception {
     Logger.getRootLogger().setLevel(Level.WARN);
     
-    String server = args[0];
-    int timeout = 30 * 1000;
     InputStream in = System.in;
     boolean overwrite = false;
     if (args.length > 1) {
@@ -109,14 +105,9 @@ public class RestoreZookeeper {
       if (arg.equals("--overwrite"))
         overwrite = true;
     
-    ZooKeeper zk = new ZooKeeper(server, timeout, new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {}
-    });
-    
     SAXParserFactory factory = SAXParserFactory.newInstance();
     SAXParser parser = factory.newSAXParser();
-    parser.parse(in, new Restore(zk, overwrite));
+    parser.parse(in, new Restore(ZooReaderWriter.getInstance(), overwrite));
     in.close();
   }
 }



Mime
View raw message