You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2013/05/21 04:12:29 UTC

svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Author: cjnolet
Date: Tue May 21 02:12:28 2013
New Revision: 1484644

URL: http://svn.apache.org/r1484644
Log:
ACCUMULO-1438 Moving MiniAccumuloCluster into mini module

Added:
    accumulo/branches/1.5/mini/   (with props)
    accumulo/branches/1.5/mini/pom.xml   (with props)
    accumulo/branches/1.5/mini/src/
    accumulo/branches/1.5/mini/src/main/
    accumulo/branches/1.5/mini/src/main/java/
    accumulo/branches/1.5/mini/src/main/java/org/
    accumulo/branches/1.5/mini/src/main/java/org/apache/
    accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/
    accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/
    accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloCluster.java
    accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java
    accumulo/branches/1.5/mini/src/test/
    accumulo/branches/1.5/mini/src/test/java/
    accumulo/branches/1.5/mini/src/test/java/org/
    accumulo/branches/1.5/mini/src/test/java/org/apache/
    accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/
    accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/
    accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java
    accumulo/branches/1.5/mini/src/test/resources/
    accumulo/branches/1.5/mini/src/test/resources/FooFilter.jar   (with props)
    accumulo/branches/1.5/mini/src/test/resources/log4j.properties
Removed:
    accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/
    accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/
    accumulo/branches/1.5/server/src/test/resources/FooFilter.jar
Modified:
    accumulo/branches/1.5/README
    accumulo/branches/1.5/pom.xml
    accumulo/branches/1.5/proxy/pom.xml
    accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
    accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
    accumulo/branches/1.5/server/src/test/resources/log4j.properties
    accumulo/branches/1.5/test/pom.xml
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java

Modified: accumulo/branches/1.5/README
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/README?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/README (original)
+++ accumulo/branches/1.5/README Tue May 21 02:12:28 2013
@@ -295,7 +295,7 @@ The public accumulo API is composed of :
   
  * everything under org.apache.accumulo.core.client, excluding impl packages  
  * Key, Mutation, Value, and Range  in org.apache.accumulo.core.data.
- * org.apache.accumulo.server.mini  
+ * org.apache.accumulo.mini
  
 To get started using accumulo review the example and the javadoc for the
 packages and classes mentioned above. 

Propchange: accumulo/branches/1.5/mini/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Tue May 21 02:12:28 2013
@@ -0,0 +1,7 @@
+target
+
+.project
+
+.settings
+
+.classpath

Added: accumulo/branches/1.5/mini/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/pom.xml?rev=1484644&view=auto
==============================================================================
Binary file - no diff available.

Propchange: accumulo/branches/1.5/mini/pom.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Added: accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloCluster.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloCluster.java?rev=1484644&view=auto
==============================================================================
--- accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloCluster.java (added)
+++ accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloCluster.java Tue May 21 02:12:28 2013
@@ -0,0 +1 @@
+/*
 * 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.mini;

import java.io.BufferedReader;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;

import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.util.UtilWaitThread;
import org.apache.accumulo.server.master.Master;
import org.apache.accumulo.server.tabletserver.TabletServer;
import org.apache.accumulo.server.util.Initialize;
import org.apache.accumulo.server.util.PortUtils;
import org.apache.accumulo.server.util.time.SimpleTimer;
import org.apache.accumulo.start.Main;
import org.apache.zookeeper.server.ZooKeeperServerMain;

/**
 * A utility class that will create Zookeeper and Accumulo processes that write all of their data to a single local directory. This class makes it easy to test
 * code against a real Accumulo instance. Its much more accurate for testing than MockAccumulo, but much slower than MockAccumulo.
 *
 * @since 1.5.0
 */
public class MiniAccumuloCluster {

  private static final String INSTANCE_SECRET = "DONTTELL";
  private static final String INSTANCE_NAME = "miniInstance";

  private static class LogWriter extends Thread {
    private BufferedReader in;
    private BufferedWriter out;

    /**
     * @throws IOException
     */
    public LogWriter(InputStream stream, File logFile) throws IOException {
      this.setDaemon(true);
      this.in = new BufferedReader(new InputStreamReader(stream));
      out = new BufferedWriter(new FileWriter(logFile));

      SimpleTimer.getInstance().schedule(new Runnable() {
        @Override
        public void run() {
          try {
            flush();
          } catch (IOException e) {
            e.printStackTrace();
          }
        }
      }, 1000, 1000);
    }

    public synchronized void flush() throws IOException {
      if (out != null)
        out.flush();
    }

    @Override
    public void run() {
      String line;

      try {
        while ((line = in.readLine()) != null) {
          out.append(line);
          out.append("\n");
        }

        synchronized (this) {
          out.close();
          out = null;
          in.close();
        }

      } catch (IOException e) {
      }
    }
  }

  private File libDir;
  private File confDir;
  private File zooKeeperDir;
  private File accumuloDir;
  private File zooCfgFile;
  private File logDir;
  private File walogDir;

  private Process zooKeeperProcess;
  private Process masterProcess;

  private int zooKeeperPort;

  private List<LogWriter> logWriters = new ArrayList<MiniAccumuloCluster.LogWriter>();

  private MiniAccumuloConfig config;
  private Process[] tabletServerProcesses;

  private Process exec(Class<? extends Object> clazz, String... args) throws IOException {
    String javaHome = System.getProperty("java.home");
    String javaBin = javaHome + File.separator + "bin" + File.separator + "java";
    String classpath = System.getProperty("java.class.path");

    classpath = confDir.getAbsolutePath() + File.pathSeparator + classpath;

    String className = clazz.getCanonicalName();

    ArrayList<String> argList = new ArrayList<String>();

    argList.addAll(Arrays.asList(javaBin, "-cp", classpath, "-Xmx128m", "-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75", Main.class.getName(), className));

    argList.addAll(Arrays.asList(args));

    ProcessBuilder builder = new ProcessBuilder(argList);

    builder.environment().put("ACCUMULO_HOME", config.getDir().getAbsolutePath());
    builder.environment().put("ACCUMULO_LOG_DIR", logDir.getAbsolutePath());

    // if we're running under accumulo.start, we forward these env vars
    String env = System.getenv("HADOOP_PREFIX");
    if (env != null)
      builder.environment().put("HADOOP_PREFIX", env);
    env = System.getenv("ZOOKEEPER_HOME");
    if (env != null)
      builder.environment().put("ZOOKEEPER_HOME", env);

    Process process = builder.start();

    LogWriter lw;
    lw = new LogWriter(process.getErrorStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".err"));
    logWriters.add(lw);
    lw.start();
    lw = new LogWriter(process.getInputStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".out"));
    logWriters.add(lw);
    lw.start();

    return process;
  }

  private void appendProp(FileWriter fileWriter, Property key, String value, Map<String,String> siteConfig) throws IOException {
    appendProp(fileWriter, key.getKey(), value, siteConfig);
  }

  private void appendProp(FileWriter fileWriter, String key, String value, Map<String,String> siteConfig) throws IOException {
    if (!siteConfig.containsKey(key))
      fileWriter.append("<property><name>" + key + "</name><value>" + value + "</value></property>\n");
  }

  /**
   *
   * @param dir
   *          An empty or nonexistant temp directoy that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava,
   *          and Junit provide methods for creating temporary directories.
   * @param rootPassword
   *          Initial root password for instance.
   * @throws IOException
   */
  public MiniAccumuloCluster(File dir, String rootPassword) throws IOException {
    this(new MiniAccumuloConfig(dir, rootPassword));
  }

  /**
   * @param config
   *          initial configuration
   * @throws IOException
   */

  public MiniAccumuloCluster(MiniAccumuloConfig config) throws IOException {

    if (config.getDir().exists() && !config.getDir().isDirectory())
      throw new IllegalArgumentException("Must pass in directory, " + config.getDir() + " is a file");

    if (config.getDir().exists() && config.getDir().list().length != 0)
      throw new IllegalArgumentException("Directory " + config.getDir() + " is not empty");

    this.config = config;

    libDir = new File(config.getDir(), "lib");
    confDir = new File(config.getDir(), "conf");
    accumuloDir = new File(config.getDir(), "accumulo");
    zooKeeperDir = new File(config.getDir(), "zookeeper");
    logDir = new File(config.getDir(), "logs");
    walogDir = new File(config.getDir(), "walogs");

    confDir.mkdirs();
    accumuloDir.mkdirs();
    zooKeeperDir.mkdirs();
    logDir.mkdirs();
    walogDir.mkdirs();
    libDir.mkdirs();

    zooKeeperPort = PortUtils.getRandomFreePort();

    File siteFile = new File(confDir, "accumulo-site.xml");

    FileWriter fileWriter = new FileWriter(siteFile);
    fileWriter.append("<configuration>\n");

    HashMap<String,String> siteConfig = new HashMap<String,String>(config.getSiteConfig());

    appendProp(fileWriter, Property.INSTANCE_DFS_URI, "file:///", siteConfig);
    appendProp(fileWriter, Property.INSTANCE_DFS_DIR, accumuloDir.getAbsolutePath(), siteConfig);
    appendProp(fileWriter, Property.INSTANCE_ZK_HOST, "localhost:" + zooKeeperPort, siteConfig);
    appendProp(fileWriter, Property.INSTANCE_SECRET, INSTANCE_SECRET, siteConfig);
    appendProp(fileWriter, Property.MASTER_CLIENTPORT, "" + PortUtils.getRandomFreePort(), siteConfig);
    appendProp(fileWriter, Property.TSERV_CLIENTPORT, "" + PortUtils.getRandomFreePort(), siteConfig);
    appendProp(fileWriter, Property.TSERV_PORTSEARCH, "true", siteConfig);
    appendProp(fileWriter, Property.LOGGER_DIR, walogDir.getAbsolutePath(), siteConfig);
    appendProp(fileWriter, Property.TSERV_DATACACHE_SIZE, "10M", siteConfig);
    appendProp(fileWriter, Property.TSERV_INDEXCACHE_SIZE, "10M", siteConfig);
    appendProp(fileWriter, Property.TSERV_MAXMEM, "50M", siteConfig);
    appendProp(fileWriter, Property.TSERV_WALOG_MAX_SIZE, "100M", siteConfig);
    appendProp(fileWriter, Property.TSERV_NATIVEMAP_ENABLED, "false", siteConfig);
    appendProp(fileWriter, Property.TRACE_TOKEN_PROPERTY_PREFIX + ".password", config.getRootPassword(), siteConfig);
    appendProp(fileWriter, Property.TRACE_PORT, "" + PortUtils.getRandomFreePort(), siteConfig);
    // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5
    appendProp(fileWriter, Property.TSERV_MAJC_DELAY, "3", siteConfig);
    String cp = System.getenv("ACCUMULO_HOME")+"/lib/.*.jar,"+
            "$ZOOKEEPER_HOME/zookeeper[^.].*.jar,"+
            "$HADOOP_HOME/[^.].*.jar,"+
            "$HADOOP_HOME/lib/[^.].*.jar,"+
            "$HADOOP_PREFIX/share/hadoop/common/.*.jar," +
            "$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar," +
            "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar," +
            "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar"
            ;
    appendProp(fileWriter, Property.GENERAL_CLASSPATHS, cp, siteConfig);
    appendProp(fileWriter, Property.GENERAL_DYNAMIC_CLASSPATHS, libDir.getAbsolutePath(), siteConfig);

    for (Entry<String,String> entry : siteConfig.entrySet())
      fileWriter.append("<property><name>" + entry.getKey() + "</name><value>" + entry.getValue() + "</value></property>\n");
    fileWriter.append("</configuration>\n");
    fileWriter.close();

    zooCfgFile = new File(confDir, "zoo.cfg");
    fileWriter = new FileWriter(zooCfgFile);

    // zookeeper uses Properties to read its config, so use that to write in order to properly escape things like Windows paths
    Properties zooCfg = new Properties();
    zooCfg.setProperty("tickTime", "1000");
    zooCfg.setProperty("initLimit", "10");
    zooCfg.setProperty("syncLimit", "5");
    zooCfg.setProperty("clientPort", zooKeeperPort + "");
    zooCfg.setProperty("maxClientCnxns", "100");
    zooCfg.setProperty("dataDir", zooKeeperDir.getAbsolutePath());
    zooCfg.store(fileWriter, null);

    fileWriter.close();

  }

  /**
   * Starts Accumulo and Zookeeper processes. Can only be called once.
   *
   * @throws IOException
   * @throws InterruptedException
   * @throws IllegalStateException
   *           if already started
   */

  public void start() throws IOException, InterruptedException {
    if (zooKeeperProcess != null)
      throw new IllegalStateException("Already started");

    Runtime.getRuntime().addShutdownHook(new Thread() {
      @Override
      public void run() {
        try {
          MiniAccumuloCluster.this.stop();
        } catch (IOException e) {
          e.printStackTrace();
        } catch (InterruptedException e) {
          e.printStackTrace();
        }
      }
    });

    zooKeeperProcess = exec(Main.class, ZooKeeperServerMain.class.getName(), zooCfgFile.getAbsolutePath());

    // sleep a little bit to let zookeeper come up before calling init, seems to work better
    UtilWaitThread.sleep(250);

    Process initProcess = exec(Initialize.class, "--instance-name", INSTANCE_NAME, "--password", config.getRootPassword(), "--username", "root");
    int ret = initProcess.waitFor();
    if (ret != 0) {
      throw new RuntimeException("Initialize process returned " + ret);
    }

    tabletServerProcesses = new Process[config.getNumTservers()];
    for (int i = 0; i < config.getNumTservers(); i++) {
      tabletServerProcesses[i] = exec(TabletServer.class);
    }

    masterProcess = exec(Master.class);
  }

  /**
   * @return Accumulo instance name
   */

  public String getInstanceName() {
    return INSTANCE_NAME;
  }

  /**
   * @return zookeeper connection string
   */

  public String getZooKeepers() {
    return "localhost:" + zooKeeperPort;
  }

  /**
   * Stops Accumulo and Zookeeper processes. If stop is not called, there is a shutdown hook that is setup to kill the processes. Howerver its probably best to
   * call stop in a finally block as soon as possible.
   *
   * @throws IOException
   * @throws InterruptedException
   */

  public void stop() throws IOException, InterruptedException {
    if (zooKeeperProcess != null)
      zooKeeperProcess.destroy();
    if (masterProcess != null)
      masterProcess.destroy();
    if (tabletServerProcesses != null) {
      for (Process tserver : tabletServerProcesses) {
        tserver.destroy();
      }
    }

    for (LogWriter lw : logWriters)
      lw.flush();
  }
}
\ No newline at end of file

Added: accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java?rev=1484644&view=auto
==============================================================================
--- accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java (added)
+++ accumulo/branches/1.5/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java Tue May 21 02:12:28 2013
@@ -0,0 +1 @@
+/*
 * 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.mini;

import java.io.File;
import java.util.Collections;
import java.util.Map;

/**
 * Holds configuration for {@link MiniAccumuloCluster}. Required configurations must be passed to constructor and all other configurations are optional.
 *
 * @since 1.5.0
 */

public class MiniAccumuloConfig {

  private File dir = null;
  private String rootPassword = null;
  private Map<String,String> siteConfig = Collections.emptyMap();
  private int numTservers = 2;

  /**
   * @param dir
   *          An empty or nonexistant temp directoy that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava,
   *          and Junit provide methods for creating temporary directories.
   * @param rootPassword
   *          The initial password for the Accumulo root user
   */

  public MiniAccumuloConfig(File dir, String rootPassword) {
    this.dir = dir;
    this.rootPassword = rootPassword;
  }

  public File getDir() {
    return dir;
  }

  public String getRootPassword() {
    return rootPassword;
  }

  public int getNumTservers() {
    return numTservers;
  }

  /**
   * Calling this method is optional. If not set, it defaults to two.
   *
   * @param numTservers
   *          the number of tablet servers that mini accumulo cluster should start
   */

  public MiniAccumuloConfig setNumTservers(int numTservers) {
    if (numTservers < 1)
      throw new IllegalArgumentException("Must have at least one tablet server");
    this.numTservers = numTservers;
    return this;
  }

  public Map<String,String> getSiteConfig() {
    return siteConfig;
  }

  /**
   * Calling this method is optional. If not set, it defautls to an empty map.
   *
   * @param siteConfig
   *          key/values that you normally put in accumulo-site.xml can be put here
   */

  public MiniAccumuloConfig setSiteConfig(Map<String,String> siteConfig) {
    this.siteConfig = siteConfig;
    return this;
  }
}
\ No newline at end of file

Added: accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java?rev=1484644&view=auto
==============================================================================
--- accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java (added)
+++ accumulo/branches/1.5/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java Tue May 21 02:12:28 2013
@@ -0,0 +1 @@
+/*
 * 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.mini;

import java.io.File;
import java.util.Collections;
import java.util.Map.Entry;
import java.util.UUID;

import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.BatchWriterConfig;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.ZooKeeperInstance;
import org.apache.accumulo.core.client.security.tokens.PasswordToken;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.LongCombiner;
import org.apache.accumulo.core.iterators.user.SummingCombiner;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.security.TablePermission;
import org.apache.commons.io.FileUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

public class MiniAccumuloClusterTest {

  public static TemporaryFolder folder = new TemporaryFolder();

  private static MiniAccumuloCluster accumulo;

  @BeforeClass
  public static void setupMiniCluster() throws Exception {

    folder.create();

    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);

    accumulo = new MiniAccumuloCluster(folder.getRoot(), "superSecret");

    accumulo.start();

  }

  @Test(timeout = 30000)
  public void test() throws Exception {
    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", new PasswordToken("superSecret"));

    conn.tableOperations().create("table1");

    conn.securityOperations().createLocalUser("user1", new PasswordToken("pass1"));
    conn.securityOperations().changeUserAuthorizations("user1", new Authorizations("A", "B"));
    conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.WRITE);
    conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.READ);

    IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
    SummingCombiner.setEncodingType(is, LongCombiner.Type.STRING);
    SummingCombiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("META", "COUNT")));

    conn.tableOperations().attachIterator("table1", is);

    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("user1", new PasswordToken("pass1"));

    BatchWriter bw = uconn.createBatchWriter("table1", new BatchWriterConfig());

    UUID uuid = UUID.randomUUID();

    Mutation m = new Mutation(uuid.toString());
    m.put("META", "SIZE", new ColumnVisibility("A|B"), "8");
    m.put("META", "CRC", new ColumnVisibility("A|B"), "456");
    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
    m.put("DATA", "IMG", new ColumnVisibility("A&B"), "ABCDEFGH");

    bw.addMutation(m);
    bw.flush();

    m = new Mutation(uuid.toString());
    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
    m.put("META", "CRC", new ColumnVisibility("A|B"), "123");
    bw.addMutation(m);

    bw.close();

    int count = 0;
    Scanner scanner = uconn.createScanner("table1", new Authorizations("A"));
    for (Entry<Key,Value> entry : scanner) {
      if (entry.getKey().getColumnQualifierData().toString().equals("COUNT")) {
        Assert.assertEquals("2", entry.getValue().toString());
      } else if (entry.getKey().getColumnQualifierData().toString().equals("SIZE")) {
        Assert.assertEquals("8", entry.getValue().toString());
      } else if (entry.getKey().getColumnQualifierData().toString().equals("CRC")) {
        Assert.assertEquals("123", entry.getValue().toString());
      } else {
        Assert.assertTrue(false);
      }
      count++;
    }

    Assert.assertEquals(3, count);

    count = 0;
    scanner = uconn.createScanner("table1", new Authorizations("A", "B"));
    for (Entry<Key,Value> entry : scanner) {
      if (entry.getKey().getColumnQualifierData().toString().equals("IMG")) {
        Assert.assertEquals("ABCDEFGH", entry.getValue().toString());
      }
      count++;
    }

    Assert.assertEquals(4, count);

    conn.tableOperations().delete("table1");
  }

  @Test(timeout = 60000)
  public void testPerTableClasspath() throws Exception {

    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", new PasswordToken("superSecret"));

    conn.tableOperations().create("table2");

    File jarFile = File.createTempFile("iterator", ".jar");
    FileUtils.copyURLToFile(this.getClass().getResource("/FooFilter.jar"), jarFile);
    jarFile.deleteOnExit();

    conn.instanceOperations().setProperty(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1", jarFile.toURI().toString());
    conn.tableOperations().setProperty("table2", Property.TABLE_CLASSPATH.getKey(), "cx1");
    conn.tableOperations().attachIterator("table2", new IteratorSetting(100, "foocensor", "org.apache.accumulo.test.FooFilter"));

    BatchWriter bw = conn.createBatchWriter("table2", new BatchWriterConfig());

    Mutation m1 = new Mutation("foo");
    m1.put("cf1", "cq1", "v2");
    m1.put("cf1", "cq2", "v3");

    bw.addMutation(m1);

    Mutation m2 = new Mutation("bar");
    m2.put("cf1", "cq1", "v6");
    m2.put("cf1", "cq2", "v7");

    bw.addMutation(m2);

    bw.close();

    Scanner scanner = conn.createScanner("table2", new Authorizations());

    int count = 0;
    for (Entry<Key,Value> entry : scanner) {
      Assert.assertFalse(entry.getKey().getRowData().toString().toLowerCase().contains("foo"));
      count++;
    }

    Assert.assertEquals(2, count);

    conn.instanceOperations().removeProperty(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
    conn.tableOperations().delete("table2");

  }

  @AfterClass
  public static void tearDownMiniCluster() throws Exception {
    accumulo.stop();
    // folder.delete();
  }

}
\ No newline at end of file

Added: accumulo/branches/1.5/mini/src/test/resources/FooFilter.jar
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/src/test/resources/FooFilter.jar?rev=1484644&view=auto
==============================================================================
Binary file - no diff available.

Propchange: accumulo/branches/1.5/mini/src/test/resources/FooFilter.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: accumulo/branches/1.5/mini/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/mini/src/test/resources/log4j.properties?rev=1484644&view=auto
==============================================================================
--- accumulo/branches/1.5/mini/src/test/resources/log4j.properties (added)
+++ accumulo/branches/1.5/mini/src/test/resources/log4j.properties Tue May 21 02:12:28 2013
@@ -0,0 +1 @@
+# 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.

log4j.rootLogger=INFO, CA
log4j.appender.CA=org.apache.log4j.ConsoleAppender
log4j.appender.CA.layout=org.apache.log4j.PatternLayout
log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n

log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
\ No newline at end of file

Modified: accumulo/branches/1.5/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/pom.xml?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/pom.xml (original)
+++ accumulo/branches/1.5/pom.xml Tue May 21 02:12:28 2013
@@ -81,6 +81,7 @@
     <module>assemble</module>
     <module>proxy</module>
     <module>test</module>
+    <module>mini</module>
   </modules>
   <scm>
     <connection>scm:svn:http://svn.apache.org/repos/asf/accumulo/branches/1.5</connection>
@@ -219,6 +220,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.accumulo</groupId>
+        <artifactId>accumulo-mini</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.accumulo</groupId>
         <artifactId>accumulo-proxy</artifactId>
         <version>${project.version}</version>
       </dependency>

Modified: accumulo/branches/1.5/proxy/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/pom.xml?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/pom.xml (original)
+++ accumulo/branches/1.5/proxy/pom.xml Tue May 21 02:12:28 2013
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-server</artifactId>
+      <artifactId>accumulo-mini</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.thrift</groupId>

Modified: accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java (original)
+++ accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java Tue May 21 02:12:28 2013
@@ -25,7 +25,7 @@ import java.util.Properties;
 
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TCompactProtocol;

Modified: accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java (original)
+++ accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java Tue May 21 02:12:28 2013
@@ -80,7 +80,7 @@ import org.apache.accumulo.proxy.thrift.
 import org.apache.accumulo.proxy.thrift.UnknownScanner;
 import org.apache.accumulo.proxy.thrift.UnknownWriter;
 import org.apache.accumulo.proxy.thrift.WriterOptions;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.commons.io.FileUtils;

Modified: accumulo/branches/1.5/server/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/test/resources/log4j.properties?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/test/resources/log4j.properties (original)
+++ accumulo/branches/1.5/server/src/test/resources/log4j.properties Tue May 21 02:12:28 2013
@@ -19,4 +19,3 @@ log4j.appender.CA.layout=org.apache.log4
 log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 
 log4j.logger.org.apache.accumulo.server.util.TabletIterator=ERROR
-log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR

Modified: accumulo/branches/1.5/test/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/pom.xml?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/test/pom.xml (original)
+++ accumulo/branches/1.5/test/pom.xml Tue May 21 02:12:28 2013
@@ -43,6 +43,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-mini</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-server</artifactId>
     </dependency>
     <dependency>

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java Tue May 21 02:12:28 2013
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.accumulo.fate.zookeeper.ZooLock.AsyncLockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Tue May 21 02:12:28 2013
@@ -27,8 +27,8 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
-import org.apache.accumulo.server.mini.MiniAccumuloConfig;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloConfig;
 import org.apache.hadoop.io.Text;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Tue May 21 02:12:28 2013
@@ -47,8 +47,8 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.shell.Shell;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
-import org.apache.accumulo.server.mini.MiniAccumuloConfig;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloConfig;
 import org.apache.accumulo.server.trace.TraceServer;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java?rev=1484644&r1=1484643&r2=1484644&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java Tue May 21 02:12:28 2013
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.mini.MiniAccumuloCluster;
 import org.apache.hadoop.io.Text;
 import org.junit.After;
 import org.junit.Before;



Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
BTW

revision can be reverted w/ a negative.   For example to revert revision 9
and 4 could do the following

svn merge -c -9 .
svn merge -c -4 .




On Tue, May 21, 2013 at 10:23 AM, Keith Turner <ke...@deenlo.com> wrote:

> Corey,
>
> It seems these move were not done is such a way that history was
> preserved.   'svn move' should be used, it preserves history.   Although I
> never use this.  When I move things around in eclipse it automatically does
> 'svn move'.
>
> I looked into how to fix this.  There does not seem to be an easy way
> other than reverting the changes and doing a move.
>
> Keith
>
>
>
> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>
>> Author: cjnolet
>> Date: Tue May 21 02:12:28 2013
>> New Revision: 1484644
>>
>> URL: http://svn.apache.org/r1484644
>> Log:
>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>>
>>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:

> Ill revert this and redo. Still getting used to svn.
>

Yeah, svn is quirky w/ this type of stuff, I am still learning too when it
comes to these corner cases.

I just did an experiment on my local 1.5  I did the following..


svn merge -c -1484792 .
svn merge -c -1484644 .
resolve --accept=working mini

svn status shows the following.  I looked at the docs, the '+' means that
history changes are scheduled.  So I think this means the history will come
back, but not positive. Not completely sure I resolved the tree conflict
correctly.

svn status
M       test/compat/japi-compliance/japi-accumulo-1.4.xml
M       test/system/upgrade_test.sh
M
test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
M       test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
M
test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
M       test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
M       test/pom.xml
M       proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
M       proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
M       proxy/pom.xml
M       pom.xml
M       assemble/pom.xml
M       mini/pom.xml
M       README
A  +    server/src/test/java/org/apache/accumulo/server/mini
A  +
 server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
A  +    server/src/test/resources/FooFilter.jar
M       server/src/test/resources/log4j.properties
A  +    server/src/main/java/org/apache/accumulo/server/mini
A  +
 server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
A  +
 server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java

I am thinking it would be best to revert in each branch.  Then merge from
1.5 to 1.6 w/o making changes, and just commit the props that mark the
merge as done.



> Corey,
>
> It seems these move were not done is such a way that history was preserved.
>   'svn move' should be used, it preserves history.   Although I never use
> this.  When I move things around in eclipse it automatically does 'svn
> move'.
>
> I looked into how to fix this.  There does not seem to be an easy way other
> than reverting the changes and doing a move.
>
> Keith
>
>
> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>
> > Author: cjnolet
> > Date: Tue May 21 02:12:28 2013
> > New Revision: 1484644
> >
> > URL: http://svn.apache.org/r1484644
> > Log:
> > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Christopher <ct...@apache.org>.
On Tue, May 21, 2013 at 11:58 AM, Keith Turner <ke...@deenlo.com> wrote:
> On Tue, May 21, 2013 at 11:54 AM, Corey Nolet <cj...@gmail.com> wrote:
>
>> Doing "svn merge -r 1484792:1484643 ." seemed to reverse the two commits
>> correctly without any conflicts (it correctly removed the mini/ directory
>> as well). I think this worked.
>>
>
> I suppose only the mini move commits fall in that range?

Hopefully... otherwise, the alternative to reverse a merge is to do:
svn merge -c -ZZZZZZ,-YYYYYY,-XXXXXX
(note the dash prior to each rev, and that the revs are in descending
order... or ascending towards zero, if you treat the dash as a
negative sign)

>>
>>
>> On Tue, May 21, 2013 at 11:40 AM, Keith Turner <ke...@deenlo.com> wrote:
>>
>> > On Tue, May 21, 2013 at 11:37 AM, Corey Nolet <cj...@gmail.com> wrote:
>> >
>> > > I just did a revert locally. In your svn status above, was there a
>> reason
>> > > for not doing an "svn delete mini" to get rid of the mini directory
>> that
>> > > was created?
>> > >
>> >
>> > No, I was not sure what was going on w/ that directory.  I knew there was
>> > something screwy, but I did not want to bother figuring out what I should
>> > do if you were going to work on it.
>> >
>> > If you get this to work.. I would be interested in seeing what svn
>> commands
>> > you used.
>> >
>> >
>> > >
>> > >
>> > > On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com>
>> wrote:
>> > >
>> > > > I'd like to try it if you don't mind. The only way to learn is to do.
>> > > >
>> > > > I'll hold on committing the change until i'm 100% confident that I'm
>> > not
>> > > > going to make the problem worse.
>> > > >
>> > > >
>> > > > On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com>
>> > wrote:
>> > > >
>> > > >> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
>> > > wrote:
>> > > >>
>> > > >> > Ill revert this and redo. Still getting used to svn.
>> > > >> >
>> > > >>
>> > > >> let me know if you want me to do the revert.
>> > > >>
>> > > >>
>> > > >> > Corey,
>> > > >> >
>> > > >> > It seems these move were not done is such a way that history was
>> > > >> preserved.
>> > > >> >   'svn move' should be used, it preserves history.   Although I
>> > never
>> > > >> use
>> > > >> > this.  When I move things around in eclipse it automatically does
>> > 'svn
>> > > >> > move'.
>> > > >> >
>> > > >> > I looked into how to fix this.  There does not seem to be an easy
>> > way
>> > > >> other
>> > > >> > than reverting the changes and doing a move.
>> > > >> >
>> > > >> > Keith
>> > > >> >
>> > > >> >
>> > > >> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>> > > >> >
>> > > >> > > Author: cjnolet
>> > > >> > > Date: Tue May 21 02:12:28 2013
>> > > >> > > New Revision: 1484644
>> > > >> > >
>> > > >> > > URL: http://svn.apache.org/r1484644
>> > > >> > > Log:
>> > > >> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>> > > >> > >
>> > > >> > >
>> > > >> >
>> > > >>
>> > > >
>> > > >
>> > >
>> >
>>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 11:54 AM, Corey Nolet <cj...@gmail.com> wrote:

> Doing "svn merge -r 1484792:1484643 ." seemed to reverse the two commits
> correctly without any conflicts (it correctly removed the mini/ directory
> as well). I think this worked.
>

I suppose only the mini move commits fall in that range?


>
>
> On Tue, May 21, 2013 at 11:40 AM, Keith Turner <ke...@deenlo.com> wrote:
>
> > On Tue, May 21, 2013 at 11:37 AM, Corey Nolet <cj...@gmail.com> wrote:
> >
> > > I just did a revert locally. In your svn status above, was there a
> reason
> > > for not doing an "svn delete mini" to get rid of the mini directory
> that
> > > was created?
> > >
> >
> > No, I was not sure what was going on w/ that directory.  I knew there was
> > something screwy, but I did not want to bother figuring out what I should
> > do if you were going to work on it.
> >
> > If you get this to work.. I would be interested in seeing what svn
> commands
> > you used.
> >
> >
> > >
> > >
> > > On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com>
> wrote:
> > >
> > > > I'd like to try it if you don't mind. The only way to learn is to do.
> > > >
> > > > I'll hold on committing the change until i'm 100% confident that I'm
> > not
> > > > going to make the problem worse.
> > > >
> > > >
> > > > On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com>
> > wrote:
> > > >
> > > >> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
> > > wrote:
> > > >>
> > > >> > Ill revert this and redo. Still getting used to svn.
> > > >> >
> > > >>
> > > >> let me know if you want me to do the revert.
> > > >>
> > > >>
> > > >> > Corey,
> > > >> >
> > > >> > It seems these move were not done is such a way that history was
> > > >> preserved.
> > > >> >   'svn move' should be used, it preserves history.   Although I
> > never
> > > >> use
> > > >> > this.  When I move things around in eclipse it automatically does
> > 'svn
> > > >> > move'.
> > > >> >
> > > >> > I looked into how to fix this.  There does not seem to be an easy
> > way
> > > >> other
> > > >> > than reverting the changes and doing a move.
> > > >> >
> > > >> > Keith
> > > >> >
> > > >> >
> > > >> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> > > >> >
> > > >> > > Author: cjnolet
> > > >> > > Date: Tue May 21 02:12:28 2013
> > > >> > > New Revision: 1484644
> > > >> > >
> > > >> > > URL: http://svn.apache.org/r1484644
> > > >> > > Log:
> > > >> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Corey Nolet <cj...@gmail.com>.
Doing "svn merge -r 1484792:1484643 ." seemed to reverse the two commits
correctly without any conflicts (it correctly removed the mini/ directory
as well). I think this worked.


On Tue, May 21, 2013 at 11:40 AM, Keith Turner <ke...@deenlo.com> wrote:

> On Tue, May 21, 2013 at 11:37 AM, Corey Nolet <cj...@gmail.com> wrote:
>
> > I just did a revert locally. In your svn status above, was there a reason
> > for not doing an "svn delete mini" to get rid of the mini directory that
> > was created?
> >
>
> No, I was not sure what was going on w/ that directory.  I knew there was
> something screwy, but I did not want to bother figuring out what I should
> do if you were going to work on it.
>
> If you get this to work.. I would be interested in seeing what svn commands
> you used.
>
>
> >
> >
> > On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com> wrote:
> >
> > > I'd like to try it if you don't mind. The only way to learn is to do.
> > >
> > > I'll hold on committing the change until i'm 100% confident that I'm
> not
> > > going to make the problem worse.
> > >
> > >
> > > On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com>
> wrote:
> > >
> > >> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
> > wrote:
> > >>
> > >> > Ill revert this and redo. Still getting used to svn.
> > >> >
> > >>
> > >> let me know if you want me to do the revert.
> > >>
> > >>
> > >> > Corey,
> > >> >
> > >> > It seems these move were not done is such a way that history was
> > >> preserved.
> > >> >   'svn move' should be used, it preserves history.   Although I
> never
> > >> use
> > >> > this.  When I move things around in eclipse it automatically does
> 'svn
> > >> > move'.
> > >> >
> > >> > I looked into how to fix this.  There does not seem to be an easy
> way
> > >> other
> > >> > than reverting the changes and doing a move.
> > >> >
> > >> > Keith
> > >> >
> > >> >
> > >> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> > >> >
> > >> > > Author: cjnolet
> > >> > > Date: Tue May 21 02:12:28 2013
> > >> > > New Revision: 1484644
> > >> > >
> > >> > > URL: http://svn.apache.org/r1484644
> > >> > > Log:
> > >> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > >> > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 11:37 AM, Corey Nolet <cj...@gmail.com> wrote:

> I just did a revert locally. In your svn status above, was there a reason
> for not doing an "svn delete mini" to get rid of the mini directory that
> was created?
>

No, I was not sure what was going on w/ that directory.  I knew there was
something screwy, but I did not want to bother figuring out what I should
do if you were going to work on it.

If you get this to work.. I would be interested in seeing what svn commands
you used.


>
>
> On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com> wrote:
>
> > I'd like to try it if you don't mind. The only way to learn is to do.
> >
> > I'll hold on committing the change until i'm 100% confident that I'm not
> > going to make the problem worse.
> >
> >
> > On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com> wrote:
> >
> >> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
> wrote:
> >>
> >> > Ill revert this and redo. Still getting used to svn.
> >> >
> >>
> >> let me know if you want me to do the revert.
> >>
> >>
> >> > Corey,
> >> >
> >> > It seems these move were not done is such a way that history was
> >> preserved.
> >> >   'svn move' should be used, it preserves history.   Although I never
> >> use
> >> > this.  When I move things around in eclipse it automatically does 'svn
> >> > move'.
> >> >
> >> > I looked into how to fix this.  There does not seem to be an easy way
> >> other
> >> > than reverting the changes and doing a move.
> >> >
> >> > Keith
> >> >
> >> >
> >> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> >> >
> >> > > Author: cjnolet
> >> > > Date: Tue May 21 02:12:28 2013
> >> > > New Revision: 1484644
> >> > >
> >> > > URL: http://svn.apache.org/r1484644
> >> > > Log:
> >> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> >> > >
> >> > >
> >> >
> >>
> >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Corey Nolet <cj...@gmail.com>.
I just did a revert locally. In your svn status above, was there a reason
for not doing an "svn delete mini" to get rid of the mini directory that
was created?


On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com> wrote:

> I'd like to try it if you don't mind. The only way to learn is to do.
>
> I'll hold on committing the change until i'm 100% confident that I'm not
> going to make the problem worse.
>
>
> On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com> wrote:
>
>> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
>>
>> > Ill revert this and redo. Still getting used to svn.
>> >
>>
>> let me know if you want me to do the revert.
>>
>>
>> > Corey,
>> >
>> > It seems these move were not done is such a way that history was
>> preserved.
>> >   'svn move' should be used, it preserves history.   Although I never
>> use
>> > this.  When I move things around in eclipse it automatically does 'svn
>> > move'.
>> >
>> > I looked into how to fix this.  There does not seem to be an easy way
>> other
>> > than reverting the changes and doing a move.
>> >
>> > Keith
>> >
>> >
>> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>> >
>> > > Author: cjnolet
>> > > Date: Tue May 21 02:12:28 2013
>> > > New Revision: 1484644
>> > >
>> > > URL: http://svn.apache.org/r1484644
>> > > Log:
>> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>> > >
>> > >
>> >
>>
>
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 11:19 AM, Corey Nolet <cj...@gmail.com> wrote:

> I'd like to try it if you don't mind. The only way to learn is to do.
>

go for it


>
> I'll hold on committing the change until i'm 100% confident that I'm not
> going to make the problem worse.
>
>
> On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com> wrote:
>
> > On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
> >
> > > Ill revert this and redo. Still getting used to svn.
> > >
> >
> > let me know if you want me to do the revert.
> >
> >
> > > Corey,
> > >
> > > It seems these move were not done is such a way that history was
> > preserved.
> > >   'svn move' should be used, it preserves history.   Although I never
> use
> > > this.  When I move things around in eclipse it automatically does 'svn
> > > move'.
> > >
> > > I looked into how to fix this.  There does not seem to be an easy way
> > other
> > > than reverting the changes and doing a move.
> > >
> > > Keith
> > >
> > >
> > > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> > >
> > > > Author: cjnolet
> > > > Date: Tue May 21 02:12:28 2013
> > > > New Revision: 1484644
> > > >
> > > > URL: http://svn.apache.org/r1484644
> > > > Log:
> > > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > > >
> > > >
> > >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Corey Nolet <cj...@gmail.com>.
I'd like to try it if you don't mind. The only way to learn is to do.

I'll hold on committing the change until i'm 100% confident that I'm not
going to make the problem worse.


On Tue, May 21, 2013 at 11:13 AM, Keith Turner <ke...@deenlo.com> wrote:

> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
>
> > Ill revert this and redo. Still getting used to svn.
> >
>
> let me know if you want me to do the revert.
>
>
> > Corey,
> >
> > It seems these move were not done is such a way that history was
> preserved.
> >   'svn move' should be used, it preserves history.   Although I never use
> > this.  When I move things around in eclipse it automatically does 'svn
> > move'.
> >
> > I looked into how to fix this.  There does not seem to be an easy way
> other
> > than reverting the changes and doing a move.
> >
> > Keith
> >
> >
> > On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> >
> > > Author: cjnolet
> > > Date: Tue May 21 02:12:28 2013
> > > New Revision: 1484644
> > >
> > > URL: http://svn.apache.org/r1484644
> > > Log:
> > > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > >
> > >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:

> Ill revert this and redo. Still getting used to svn.
>

let me know if you want me to do the revert.


> Corey,
>
> It seems these move were not done is such a way that history was preserved.
>   'svn move' should be used, it preserves history.   Although I never use
> this.  When I move things around in eclipse it automatically does 'svn
> move'.
>
> I looked into how to fix this.  There does not seem to be an easy way other
> than reverting the changes and doing a move.
>
> Keith
>
>
> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>
> > Author: cjnolet
> > Date: Tue May 21 02:12:28 2013
> > New Revision: 1484644
> >
> > URL: http://svn.apache.org/r1484644
> > Log:
> > ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> >
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 12:09 PM, Corey Nolet <cj...@gmail.com> wrote:

> Keith: Correct. If you specify the revisions in reverse order, it reverses
> those commits.
>
> I'm comfortable with this change and ready to push up. Here's what I'll do:
>
> Revert changes in 1.4.4
> Revert changes in 1.5
> Revert changes in 1.6
> Merge 1.5 to 1.6
>
> Is this what you were describing above Keith?
>

slightly unrelated.   We are trying to keep 1.5 and 1.6 in a state such
that "svn merge -r 1:HEAD 1.5 1.6" will work nicely.  The point of this is
to ensure that bug fixes made in 1.5 are also made in 1.6.


>
>
> On Tue, May 21, 2013 at 12:03 PM, Christopher <ct...@apache.org> wrote:
>
> > git-svn is likely to do worse with preserving svn history.
> >
> > --
> > Christopher L Tubbs II
> > http://gravatar.com/ctubbsii
> >
> >
> > On Tue, May 21, 2013 at 11:05 AM, Vincent Russell
> > <vi...@gmail.com> wrote:
> > > Corey,
> > >
> > > You should try git-svn.  That should give you an interface that you are
> > used to.
> > >
> > > On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
> wrote:
> > >> Ill revert this and redo. Still getting used to svn.
> > >> Corey,
> > >>
> > >> It seems these move were not done is such a way that history was
> > preserved.
> > >>   'svn move' should be used, it preserves history.   Although I never
> > use
> > >> this.  When I move things around in eclipse it automatically does 'svn
> > >> move'.
> > >>
> > >> I looked into how to fix this.  There does not seem to be an easy way
> > other
> > >> than reverting the changes and doing a move.
> > >>
> > >> Keith
> > >>
> > >>
> > >> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> > >>
> > >>> Author: cjnolet
> > >>> Date: Tue May 21 02:12:28 2013
> > >>> New Revision: 1484644
> > >>>
> > >>> URL: http://svn.apache.org/r1484644
> > >>> Log:
> > >>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > >>>
> > >>>
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
On Tue, May 21, 2013 at 12:09 PM, Corey Nolet <cj...@gmail.com> wrote:

> Keith: Correct. If you specify the revisions in reverse order, it reverses
> those commits.
>
> I'm comfortable with this change and ready to push up. Here's what I'll do:
>
> Revert changes in 1.4.4
> Revert changes in 1.5
> Revert changes in 1.6
> Merge 1.5 to 1.6
>
> Is this what you were describing above Keith?
>

That is what I was thinking.


>
>
> On Tue, May 21, 2013 at 12:03 PM, Christopher <ct...@apache.org> wrote:
>
> > git-svn is likely to do worse with preserving svn history.
> >
> > --
> > Christopher L Tubbs II
> > http://gravatar.com/ctubbsii
> >
> >
> > On Tue, May 21, 2013 at 11:05 AM, Vincent Russell
> > <vi...@gmail.com> wrote:
> > > Corey,
> > >
> > > You should try git-svn.  That should give you an interface that you are
> > used to.
> > >
> > > On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com>
> wrote:
> > >> Ill revert this and redo. Still getting used to svn.
> > >> Corey,
> > >>
> > >> It seems these move were not done is such a way that history was
> > preserved.
> > >>   'svn move' should be used, it preserves history.   Although I never
> > use
> > >> this.  When I move things around in eclipse it automatically does 'svn
> > >> move'.
> > >>
> > >> I looked into how to fix this.  There does not seem to be an easy way
> > other
> > >> than reverting the changes and doing a move.
> > >>
> > >> Keith
> > >>
> > >>
> > >> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> > >>
> > >>> Author: cjnolet
> > >>> Date: Tue May 21 02:12:28 2013
> > >>> New Revision: 1484644
> > >>>
> > >>> URL: http://svn.apache.org/r1484644
> > >>> Log:
> > >>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> > >>>
> > >>>
> >
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Corey Nolet <cj...@gmail.com>.
Keith: Correct. If you specify the revisions in reverse order, it reverses
those commits.

I'm comfortable with this change and ready to push up. Here's what I'll do:

Revert changes in 1.4.4
Revert changes in 1.5
Revert changes in 1.6
Merge 1.5 to 1.6

Is this what you were describing above Keith?


On Tue, May 21, 2013 at 12:03 PM, Christopher <ct...@apache.org> wrote:

> git-svn is likely to do worse with preserving svn history.
>
> --
> Christopher L Tubbs II
> http://gravatar.com/ctubbsii
>
>
> On Tue, May 21, 2013 at 11:05 AM, Vincent Russell
> <vi...@gmail.com> wrote:
> > Corey,
> >
> > You should try git-svn.  That should give you an interface that you are
> used to.
> >
> > On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
> >> Ill revert this and redo. Still getting used to svn.
> >> Corey,
> >>
> >> It seems these move were not done is such a way that history was
> preserved.
> >>   'svn move' should be used, it preserves history.   Although I never
> use
> >> this.  When I move things around in eclipse it automatically does 'svn
> >> move'.
> >>
> >> I looked into how to fix this.  There does not seem to be an easy way
> other
> >> than reverting the changes and doing a move.
> >>
> >> Keith
> >>
> >>
> >> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
> >>
> >>> Author: cjnolet
> >>> Date: Tue May 21 02:12:28 2013
> >>> New Revision: 1484644
> >>>
> >>> URL: http://svn.apache.org/r1484644
> >>> Log:
> >>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
> >>>
> >>>
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Christopher <ct...@apache.org>.
git-svn is likely to do worse with preserving svn history.

--
Christopher L Tubbs II
http://gravatar.com/ctubbsii


On Tue, May 21, 2013 at 11:05 AM, Vincent Russell
<vi...@gmail.com> wrote:
> Corey,
>
> You should try git-svn.  That should give you an interface that you are used to.
>
> On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
>> Ill revert this and redo. Still getting used to svn.
>> Corey,
>>
>> It seems these move were not done is such a way that history was preserved.
>>   'svn move' should be used, it preserves history.   Although I never use
>> this.  When I move things around in eclipse it automatically does 'svn
>> move'.
>>
>> I looked into how to fix this.  There does not seem to be an easy way other
>> than reverting the changes and doing a move.
>>
>> Keith
>>
>>
>> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>>
>>> Author: cjnolet
>>> Date: Tue May 21 02:12:28 2013
>>> New Revision: 1484644
>>>
>>> URL: http://svn.apache.org/r1484644
>>> Log:
>>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>>>
>>>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Vincent Russell <vi...@gmail.com>.
Corey,

You should try git-svn.  That should give you an interface that you are used to.

On Tue, May 21, 2013 at 10:53 AM, Corey Nolet <cj...@gmail.com> wrote:
> Ill revert this and redo. Still getting used to svn.
> Corey,
>
> It seems these move were not done is such a way that history was preserved.
>   'svn move' should be used, it preserves history.   Although I never use
> this.  When I move things around in eclipse it automatically does 'svn
> move'.
>
> I looked into how to fix this.  There does not seem to be an easy way other
> than reverting the changes and doing a move.
>
> Keith
>
>
> On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:
>
>> Author: cjnolet
>> Date: Tue May 21 02:12:28 2013
>> New Revision: 1484644
>>
>> URL: http://svn.apache.org/r1484644
>> Log:
>> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>>
>>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Corey Nolet <cj...@gmail.com>.
Ill revert this and redo. Still getting used to svn.
Corey,

It seems these move were not done is such a way that history was preserved.
  'svn move' should be used, it preserves history.   Although I never use
this.  When I move things around in eclipse it automatically does 'svn
move'.

I looked into how to fix this.  There does not seem to be an easy way other
than reverting the changes and doing a move.

Keith


On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:

> Author: cjnolet
> Date: Tue May 21 02:12:28 2013
> New Revision: 1484644
>
> URL: http://svn.apache.org/r1484644
> Log:
> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>
>

Re: svn commit: r1484644 - in /accumulo/branches/1.5: ./ mini/ mini/src/ mini/src/main/ mini/src/main/java/ mini/src/main/java/org/ mini/src/main/java/org/apache/ mini/src/main/java/org/apache/accumulo/ mini/src/main/java/org/apache/accumulo/mini/ mini/src...

Posted by Keith Turner <ke...@deenlo.com>.
Corey,

It seems these move were not done is such a way that history was preserved.
  'svn move' should be used, it preserves history.   Although I never use
this.  When I move things around in eclipse it automatically does 'svn
move'.

I looked into how to fix this.  There does not seem to be an easy way other
than reverting the changes and doing a move.

Keith


On Mon, May 20, 2013 at 10:12 PM, <cj...@apache.org> wrote:

> Author: cjnolet
> Date: Tue May 21 02:12:28 2013
> New Revision: 1484644
>
> URL: http://svn.apache.org/r1484644
> Log:
> ACCUMULO-1438 Moving MiniAccumuloCluster into mini module
>
>