You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2015/05/20 18:55:59 UTC

[1/4] accumulo git commit: ACCUMULO-3000: Added tests for "existing ZooKeeper" functionality

Repository: accumulo
Updated Branches:
  refs/heads/master a9d1ad4a7 -> acf60826e


ACCUMULO-3000: Added tests for "existing ZooKeeper" functionality


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/0c069d1f
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/0c069d1f
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/0c069d1f

Branch: refs/heads/master
Commit: 0c069d1f5003fd309432765c863fb47c066a1d33
Parents: dd55a16
Author: Trent Nadeau <ta...@gmail.com>
Authored: Tue Apr 21 10:47:11 2015 -0400
Committer: Trent Nadeau <ta...@gmail.com>
Committed: Fri May 15 15:19:01 2015 -0400

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 minicluster/pom.xml                             |  10 ++
 ...niAccumuloClusterExistingZooKeepersTest.java | 107 +++++++++++++++++++
 pom.xml                                         |  12 +++
 4 files changed, 130 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0c069d1f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 56204d2..f7db771 100644
--- a/.gitignore
+++ b/.gitignore
@@ -23,3 +23,4 @@
 /.pydevproject
 /.idea
 /*.iml
+.*.swp

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0c069d1f/minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index ff390f9..afd459c 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -105,6 +105,16 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-framework</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.easymock</groupId>
       <artifactId>easymock</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0c069d1f/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
new file mode 100644
index 0000000..d9fb63e
--- /dev/null
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.minicluster;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiniAccumuloClusterExistingZooKeepersTest {
+  private static final File BASE_DIR = new File(
+          System.getProperty("user.dir") + "/target/mini-tests/" + MiniAccumuloClusterExistingZooKeepersTest.class.getName());
+
+  private static final String SECRET = "superSecret";
+
+  private static final Logger log = LoggerFactory.getLogger(MiniAccumuloClusterExistingZooKeepersTest.class);
+  private TestingServer zooKeeper;
+  private MiniAccumuloCluster accumulo;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setupTestCluster() throws Exception {
+    assertTrue(BASE_DIR.mkdirs() || BASE_DIR.isDirectory());
+    File testDir = new File(BASE_DIR, testName.getMethodName());
+    FileUtils.deleteQuietly(testDir);
+    assertTrue(testDir.mkdir());
+
+    zooKeeper = new TestingServer();
+
+    MiniAccumuloConfig config = new MiniAccumuloConfig(testDir, SECRET);
+    config.setExistingZooKeepers(zooKeeper.getConnectString());
+    accumulo = new MiniAccumuloCluster(config);
+    accumulo.start();
+  }
+
+  @After
+  public void teardownTestCluster() {
+    if (accumulo != null) {
+      try {
+        accumulo.stop();
+      } catch (IOException | InterruptedException e) {
+        log.warn("Failure during tear down", e);
+      }
+    }
+
+    if (zooKeeper != null) {
+      try {
+        zooKeeper.close();
+      } catch (IOException e) {
+        log.warn("Failure stopping test ZooKeeper server");
+      }
+    }
+  }
+
+  @Test
+  public void canConnectViaExistingZooKeeper() throws Exception {
+    Connector conn = accumulo.getConnector("root", SECRET);
+    Instance instance = conn.getInstance();
+    assertEquals(zooKeeper.getConnectString(), instance.getZooKeepers());
+
+    String tableName = "foo";
+    conn.tableOperations().create(tableName);
+    Map<String,String> tableIds = conn.tableOperations().tableIdMap();
+    assertTrue(tableIds.containsKey(tableName));
+
+    String zkTablePath = String.format("/accumulo/%s/tables/%s/name", instance.getInstanceID(), tableIds.get(tableName));
+    try (CuratorFramework client = CuratorFrameworkFactory.newClient(zooKeeper.getConnectString(), new RetryOneTime(1))) {
+      client.start();
+      assertNotNull(client.checkExists().forPath(zkTablePath));
+      assertEquals(tableName, new String(client.getData().forPath(zkTablePath)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0c069d1f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a6ac82c..4883639 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,6 +119,8 @@
     <accumulo.release.version>${project.version}</accumulo.release.version>
     <!-- bouncycastle version for test dependencies -->
     <bouncycastle.version>1.50</bouncycastle.version>
+    <!-- Curator version -->
+    <curator.version>2.7.1</curator.version>
     <!-- relative path for Eclipse format; should override in child modules if necessary -->
     <eclipseFormatterStyle>${project.parent.basedir}/contrib/Eclipse-Accumulo-Codestyle.xml</eclipseFormatterStyle>
     <!-- findbugs-maven-plugin won't work on jdk8 or later; set to 3.0.0 or newer -->
@@ -355,6 +357,16 @@
         <version>2.0</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.curator</groupId>
+        <artifactId>curator-framework</artifactId>
+        <version>${curator.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.curator</groupId>
+        <artifactId>curator-test</artifactId>
+        <version>${curator.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-client</artifactId>
         <version>${hadoop.version}</version>


[2/4] accumulo git commit: ACCUMULO-3000: Added config option and impl to use existing ZooKeeper

Posted by ct...@apache.org.
ACCUMULO-3000: Added config option and impl to use existing ZooKeeper


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/dd55a161
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/dd55a161
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/dd55a161

Branch: refs/heads/master
Commit: dd55a161bad766158ec5e876a84debeb46de2465
Parents: d353b61
Author: Trent Nadeau <ta...@gmail.com>
Authored: Thu Apr 16 14:35:54 2015 +0000
Committer: Trent Nadeau <ta...@gmail.com>
Committed: Fri May 15 15:19:01 2015 -0400

----------------------------------------------------------------------
 .../minicluster/MiniAccumuloConfig.java         | 13 +++++
 .../minicluster/MiniAccumuloRunner.java         |  4 ++
 .../impl/MiniAccumuloClusterImpl.java           | 59 +++++++++++---------
 .../impl/MiniAccumuloConfigImpl.java            | 37 ++++++++++--
 4 files changed, 83 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dd55a161/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index b71b771..932507d 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -108,6 +108,19 @@ public class MiniAccumuloConfig {
   }
 
   /**
+   * Configure an existing ZooKeeper instance to use. Calling this method is optional. If not set, a new ZooKeeper instance is created.
+   *
+   * @param existingZooKeepers
+   *          Connection string for a already-running ZooKeeper instance
+   *
+   * @since 1.7.0
+   */
+  public MiniAccumuloConfig setExistingZooKeepers(String existingZooKeepers) {
+    impl.setExistingZooKeepers(existingZooKeepers);
+    return this;
+  }
+
+  /**
    * Sets the amount of memory to use in the master process. Calling this method is optional. Default memory is 128M
    *
    * @param serverType

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dd55a161/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index 13a75b5..1f1be44 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -75,6 +75,7 @@ public class MiniAccumuloRunner {
   private static final String NUM_T_SERVERS_PROP = "numTServers";
   private static final String DIRECTORY_PROP = "directory";
   private static final String INSTANCE_NAME_PROP = "instanceName";
+  private static final String EXISTING_ZOO_KEEPERS_PROP = "existingZooKeepers";
 
   private static void printProperties() {
     System.out.println("#mini Accumulo cluster runner properties.");
@@ -93,6 +94,7 @@ public class MiniAccumuloRunner {
     System.out.println("#" + TSERVER_MEMORY_PROP + "=128M");
     System.out.println("#" + ZOO_KEEPER_MEMORY_PROP + "=128M");
     System.out.println("#" + JDWP_ENABLED_PROP + "=false");
+    System.out.println("#" + EXISTING_ZOO_KEEPERS_PROP + "=localhost:2181");
 
     System.out.println();
     System.out.println("# Configuration normally placed in accumulo-site.xml can be added using a site. prefix.");
@@ -167,6 +169,8 @@ public class MiniAccumuloRunner {
       config.setZooKeeperPort(Integer.parseInt(opts.prop.getProperty(ZOO_KEEPER_PORT_PROP)));
     if (opts.prop.containsKey(ZOO_KEEPER_STARTUP_TIME_PROP))
       config.setZooKeeperStartupTime(Long.parseLong(opts.prop.getProperty(ZOO_KEEPER_STARTUP_TIME_PROP)));
+    if (opts.prop.containsKey(EXISTING_ZOO_KEEPERS_PROP))
+      config.setExistingZooKeepers(opts.prop.getProperty(EXISTING_ZOO_KEEPERS_PROP));
     if (opts.prop.containsKey(JDWP_ENABLED_PROP))
       config.setJDWPEnabled(Boolean.parseBoolean(opts.prop.getProperty(JDWP_ENABLED_PROP)));
     if (opts.prop.containsKey(ZOO_KEEPER_MEMORY_PROP))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dd55a161/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index a21ba64..dd20850 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -375,7 +375,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     mkdirs(config.getLibExtDir());
 
     if (!config.useExistingInstance()) {
-      mkdirs(config.getZooKeeperDir());
+      if (!config.useExistingZooKeepers())
+        mkdirs(config.getZooKeeperDir());
       mkdirs(config.getWalogDir());
       mkdirs(config.getAccumuloDir());
     }
@@ -430,7 +431,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     File siteFile = new File(config.getConfDir(), "accumulo-site.xml");
     writeConfig(siteFile, config.getSiteConfig().entrySet());
 
-    if (!config.useExistingInstance()) {
+    if (!config.useExistingInstance() && !config.useExistingZooKeepers()) {
       zooCfgFile = new File(config.getConfDir(), "zoo.cfg");
       FileWriter fileWriter = new FileWriter(zooCfgFile);
 
@@ -545,32 +546,35 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         });
       }
 
-      control.start(ServerType.ZOOKEEPER);
+      if (!config.useExistingZooKeepers())
+        control.start(ServerType.ZOOKEEPER);
 
       if (!initialized) {
-        // sleep a little bit to let zookeeper come up before calling init, seems to work better
-        long startTime = System.currentTimeMillis();
-        while (true) {
-          Socket s = null;
-          try {
-            s = new Socket("localhost", config.getZooKeeperPort());
-            s.setReuseAddress(true);
-            s.getOutputStream().write("ruok\n".getBytes());
-            s.getOutputStream().flush();
-            byte buffer[] = new byte[100];
-            int n = s.getInputStream().read(buffer);
-            if (n >= 4 && new String(buffer, 0, 4).equals("imok"))
-              break;
-          } catch (Exception e) {
-            if (System.currentTimeMillis() - startTime >= config.getZooKeeperStartupTime()) {
-              throw new ZooKeeperBindException("Zookeeper did not start within " + (config.getZooKeeperStartupTime() / 1000) + " seconds. Check the logs in "
-                  + config.getLogDir() + " for errors.  Last exception: " + e);
+        if (!config.useExistingZooKeepers()) {
+          // sleep a little bit to let zookeeper come up before calling init, seems to work better
+          long startTime = System.currentTimeMillis();
+          while (true) {
+            Socket s = null;
+            try {
+              s = new Socket("localhost", config.getZooKeeperPort());
+              s.setReuseAddress(true);
+              s.getOutputStream().write("ruok\n".getBytes());
+              s.getOutputStream().flush();
+              byte buffer[] = new byte[100];
+              int n = s.getInputStream().read(buffer);
+              if (n >= 4 && new String(buffer, 0, 4).equals("imok"))
+                break;
+            } catch (Exception e) {
+              if (System.currentTimeMillis() - startTime >= config.getZooKeeperStartupTime()) {
+                throw new ZooKeeperBindException("Zookeeper did not start within " + (config.getZooKeeperStartupTime() / 1000) + " seconds. Check the logs in "
+                    + config.getLogDir() + " for errors.  Last exception: " + e);
+              }
+              // Don't spin absurdly fast
+              Thread.sleep(250);
+            } finally {
+              if (s != null)
+                s.close();
             }
-            // Don't spin absurdly fast
-            Thread.sleep(250);
-          } finally {
-            if (s != null)
-              s.close();
           }
         }
 
@@ -579,6 +583,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         args.add(config.getInstanceName());
         args.add("--user");
         args.add(config.getRootUserName());
+        args.add("--clear-instance-name");
 
         // If we aren't using SASL, add in the root password
         final String saslEnabled = config.getSiteConfig().get(Property.INSTANCE_RPC_SASL_ENABLED.getKey());
@@ -644,7 +649,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     MiniAccumuloClusterControl control = getClusterControl();
     result.put(ServerType.MASTER, references(control.masterProcess));
     result.put(ServerType.TABLET_SERVER, references(control.tabletServerProcesses.toArray(new Process[0])));
-    result.put(ServerType.ZOOKEEPER, references(control.zooKeeperProcess));
+    if (null != control.zooKeeperProcess) {
+      result.put(ServerType.ZOOKEEPER, references(control.zooKeeperProcess));
+    }
     if (null != control.gcProcess) {
       result.put(ServerType.GARBAGE_COLLECTOR, references(control.gcProcess));
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dd55a161/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index eab82ba..25f3c99 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -67,6 +67,7 @@ public class MiniAccumuloConfigImpl {
   private int zooKeeperPort = 0;
   private int configuredZooKeeperPort = 0;
   private long zooKeeperStartupTime = 20 * 1000;
+  private String existingZooKeepers;
 
   private long defaultMemorySize = 128 * 1024 * 1024;
 
@@ -157,10 +158,17 @@ public class MiniAccumuloConfigImpl {
 
       if (existingInstance == null || !existingInstance) {
         existingInstance = false;
-        // zookeeper port should be set explicitly in this class, not just on the site config
-        if (zooKeeperPort == 0)
-          zooKeeperPort = PortUtils.getRandomFreePort();
-        siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), "localhost:" + zooKeeperPort);
+        String zkHost;
+        if (useExistingZooKeepers()) {
+          zkHost = existingZooKeepers;
+        } else {
+          // zookeeper port should be set explicitly in this class, not just on the site config
+          if (zooKeeperPort == 0)
+            zooKeeperPort = PortUtils.getRandomFreePort();
+
+          zkHost = "localhost:" + zooKeeperPort;
+        }
+        siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), zkHost);
       }
       initialized = true;
     }
@@ -313,6 +321,19 @@ public class MiniAccumuloConfigImpl {
   }
 
   /**
+   * Configure an existing ZooKeeper instance to use. Calling this method is optional. If not set, a new ZooKeeper instance is created.
+   *
+   * @param existingZooKeepers
+   *          Connection string for a already-running ZooKeeper instance
+   *
+   * @since 1.8.0
+   */
+  public MiniAccumuloConfigImpl setExistingZooKeepers(String existingZooKeepers) {
+    this.existingZooKeepers = existingZooKeepers;
+    return this;
+  }
+
+  /**
    * Sets the amount of memory to use in the master process. Calling this method is optional. Default memory is 128M
    *
    * @param serverType
@@ -384,6 +405,14 @@ public class MiniAccumuloConfigImpl {
     return zooKeeperStartupTime;
   }
 
+  public String getExistingZooKeepers() {
+    return existingZooKeepers;
+  }
+
+  public boolean useExistingZooKeepers() {
+    return existingZooKeepers != null && !existingZooKeepers.isEmpty();
+  }
+
   File getLibDir() {
     return libDir;
   }


[4/4] accumulo git commit: Merge branch 'mac-with-existing-zookeeper' of github:tanadeau/accumulo

Posted by ct...@apache.org.
Merge branch 'mac-with-existing-zookeeper' of github:tanadeau/accumulo

This closes #29

Signed-off-by: Christopher Tubbs <ct...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/acf60826
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/acf60826
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/acf60826

Branch: refs/heads/master
Commit: acf60826e25f943653fd7875d3b01ef43b7adf77
Parents: a9d1ad4 6c22aab
Author: Christopher Tubbs <ct...@apache.org>
Authored: Wed May 20 12:50:38 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed May 20 12:50:38 2015 -0400

----------------------------------------------------------------------
 minicluster/pom.xml                             |  10 ++
 .../minicluster/MiniAccumuloRunner.java         |   4 +
 .../impl/MiniAccumuloClusterImpl.java           |  59 +++++-----
 .../impl/MiniAccumuloConfigImpl.java            |  37 ++++++-
 ...niAccumuloClusterExistingZooKeepersTest.java | 107 +++++++++++++++++++
 pom.xml                                         |  12 +++
 6 files changed, 199 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/acf60826/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/acf60826/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/acf60826/pom.xml
----------------------------------------------------------------------


[3/4] accumulo git commit: ACCUMULO-3000: Use MAC config impl when using existing zookeepers

Posted by ct...@apache.org.
ACCUMULO-3000: Use MAC config impl when using existing zookeepers


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/6c22aabf
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/6c22aabf
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/6c22aabf

Branch: refs/heads/master
Commit: 6c22aabfc604c6a93f215496577f9ed6ae7372bc
Parents: 0c069d1
Author: Trent Nadeau <ta...@gmail.com>
Authored: Tue May 12 17:36:38 2015 -0400
Committer: Trent Nadeau <ta...@gmail.com>
Committed: Fri May 15 15:19:02 2015 -0400

----------------------------------------------------------------------
 .../accumulo/minicluster/MiniAccumuloConfig.java       | 13 -------------
 .../accumulo/minicluster/MiniAccumuloRunner.java       |  2 +-
 .../minicluster/impl/MiniAccumuloConfigImpl.java       |  2 +-
 .../MiniAccumuloClusterExistingZooKeepersTest.java     |  6 +++---
 4 files changed, 5 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6c22aabf/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index 932507d..b71b771 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -108,19 +108,6 @@ public class MiniAccumuloConfig {
   }
 
   /**
-   * Configure an existing ZooKeeper instance to use. Calling this method is optional. If not set, a new ZooKeeper instance is created.
-   *
-   * @param existingZooKeepers
-   *          Connection string for a already-running ZooKeeper instance
-   *
-   * @since 1.7.0
-   */
-  public MiniAccumuloConfig setExistingZooKeepers(String existingZooKeepers) {
-    impl.setExistingZooKeepers(existingZooKeepers);
-    return this;
-  }
-
-  /**
    * Sets the amount of memory to use in the master process. Calling this method is optional. Default memory is 128M
    *
    * @param serverType

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6c22aabf/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index 1f1be44..78af6a0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -170,7 +170,7 @@ public class MiniAccumuloRunner {
     if (opts.prop.containsKey(ZOO_KEEPER_STARTUP_TIME_PROP))
       config.setZooKeeperStartupTime(Long.parseLong(opts.prop.getProperty(ZOO_KEEPER_STARTUP_TIME_PROP)));
     if (opts.prop.containsKey(EXISTING_ZOO_KEEPERS_PROP))
-      config.setExistingZooKeepers(opts.prop.getProperty(EXISTING_ZOO_KEEPERS_PROP));
+      config.getImpl().setExistingZooKeepers(opts.prop.getProperty(EXISTING_ZOO_KEEPERS_PROP));
     if (opts.prop.containsKey(JDWP_ENABLED_PROP))
       config.setJDWPEnabled(Boolean.parseBoolean(opts.prop.getProperty(JDWP_ENABLED_PROP)));
     if (opts.prop.containsKey(ZOO_KEEPER_MEMORY_PROP))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6c22aabf/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 25f3c99..f2c4b80 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -324,7 +324,7 @@ public class MiniAccumuloConfigImpl {
    * Configure an existing ZooKeeper instance to use. Calling this method is optional. If not set, a new ZooKeeper instance is created.
    *
    * @param existingZooKeepers
-   *          Connection string for a already-running ZooKeeper instance
+   *          Connection string for a already-running ZooKeeper instance. A null value will turn off this feature.
    *
    * @since 1.8.0
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6c22aabf/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
index d9fb63e..8c21874 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
@@ -40,8 +40,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class MiniAccumuloClusterExistingZooKeepersTest {
-  private static final File BASE_DIR = new File(
-          System.getProperty("user.dir") + "/target/mini-tests/" + MiniAccumuloClusterExistingZooKeepersTest.class.getName());
+  private static final File BASE_DIR = new File(System.getProperty("user.dir") + "/target/mini-tests/"
+      + MiniAccumuloClusterExistingZooKeepersTest.class.getName());
 
   private static final String SECRET = "superSecret";
 
@@ -62,7 +62,7 @@ public class MiniAccumuloClusterExistingZooKeepersTest {
     zooKeeper = new TestingServer();
 
     MiniAccumuloConfig config = new MiniAccumuloConfig(testDir, SECRET);
-    config.setExistingZooKeepers(zooKeeper.getConnectString());
+    config.getImpl().setExistingZooKeepers(zooKeeper.getConnectString());
     accumulo = new MiniAccumuloCluster(config);
     accumulo.start();
   }