Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 46BBCF42F for ; Wed, 17 Jul 2013 02:33:21 +0000 (UTC) Received: (qmail 8117 invoked by uid 500); 17 Jul 2013 02:33:18 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 7260 invoked by uid 500); 17 Jul 2013 02:33:07 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 6854 invoked by uid 99); 17 Jul 2013 02:33:04 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Jul 2013 02:33:04 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id CFF388AB30F; Wed, 17 Jul 2013 02:33:03 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Wed, 17 Jul 2013 02:33:27 -0000 Message-Id: <1324a0167e3a499dba4dd4dd1a53faa9@git.apache.org> In-Reply-To: <9368626895444c52a4d09631b34b56e0@git.apache.org> References: <9368626895444c52a4d09631b34b56e0@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [25/50] [abbrv] git commit: ACCUMULO-1537 completed the conversion of functional tests to IT; also converted ShellServerTest to an IT ACCUMULO-1537 completed the conversion of functional tests to IT; also converted ShellServerTest to an IT git-svn-id: https://svn.apache.org/repos/asf/accumulo/trunk@1501497 13f79535-47bb-0310-9956-ffa450edef68 Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7a1075a4 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7a1075a4 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7a1075a4 Branch: refs/heads/ACCUMULO-1496 Commit: 7a1075a47299afd6d27225b17814c11eb62432d2 Parents: 8d815a8 Author: Eric C. Newton Authored: Tue Jul 9 19:50:06 2013 +0000 Committer: Eric C. Newton Committed: Tue Jul 9 19:50:06 2013 +0000 ---------------------------------------------------------------------- .../accumulo/core/cli/ClientOnDefaultTable.java | 2 +- .../minicluster/MiniAccumuloCluster.java | 96 +- .../minicluster/ProcessNotFoundException.java | 23 + .../accumulo/minicluster/ProcessReference.java | 44 + .../server/tabletserver/TabletServer.java | 3 + .../org/apache/accumulo/test/VerifyIngest.java | 2 +- .../test/functional/CacheTestClean.java | 13 +- .../test/functional/MasterFailoverIT.java | 59 ++ .../accumulo/test/functional/NativeMapTest.java | 1 + .../test/functional/SplitRecoveryIT.java | 30 + test/src/test/c/fake_disk_failure.c | 51 ++ .../org/apache/accumulo/test/ShellServerIT.java | 865 +++++++++++++++++++ .../apache/accumulo/test/ShellServerTest.java | 805 ----------------- .../apache/accumulo/test/TableOperationsIT.java | 6 +- .../accumulo/test/functional/BinaryIT.java | 2 +- .../accumulo/test/functional/BloomFilterIT.java | 20 +- .../apache/accumulo/test/functional/BulkIT.java | 10 +- .../functional/BulkSplitOptimizationIT.java | 3 +- .../test/functional/ChaoticBlancerIT.java | 6 +- .../accumulo/test/functional/DeleteIT.java | 5 +- .../test/functional/DynamicThreadPoolsIT.java | 13 +- .../test/functional/HalfDeadTServerIT.java | 158 ++++ .../test/functional/LateLastContactIT.java | 44 + .../accumulo/test/functional/MacTest.java | 10 +- .../accumulo/test/functional/MapReduceIT.java | 2 +- .../accumulo/test/functional/MaxOpenIT.java | 2 +- .../test/functional/MetadataSplitIT.java | 50 ++ .../accumulo/test/functional/NativeMapIT.java | 30 + .../accumulo/test/functional/PermissionsIT.java | 2 +- .../accumulo/test/functional/RestartIT.java | 144 +++ .../test/functional/RestartStressIT.java | 73 ++ .../test/functional/ServerSideErrorIT.java | 2 +- .../test/functional/SparseColumnFamilyIT.java | 5 - .../accumulo/test/functional/SplitIT.java | 2 +- .../accumulo/test/functional/TableIT.java | 13 +- .../accumulo/test/functional/TimeoutIT.java | 6 +- .../accumulo/test/functional/VisibilityIT.java | 2 +- .../test/functional/WriteAheadLogIT.java | 69 ++ .../accumulo/test/functional/ZooCacheIT.java | 55 ++ test/system/auto/fake_disk_failure.c | 55 -- test/system/auto/simple/masterFailover.py | 51 -- test/system/auto/simple/nativeMap.py | 42 - test/system/auto/simple/shell.py | 474 ---------- test/system/auto/simple/splitRecovery.py | 30 - test/system/auto/simple/wal.py | 68 -- test/system/auto/simple/zoo.py | 54 -- test/system/auto/simple/zooCacheTest.py | 51 -- test/system/auto/stress/halfDead.py | 101 --- test/system/auto/stress/migrations.py | 82 -- test/system/auto/stress/msplit.py | 53 -- test/system/auto/stress/restart.py | 199 ----- test/system/auto/stress/table.py | 61 -- test/system/auto/stress/weird.py | 49 -- 53 files changed, 1839 insertions(+), 2259 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java index 44aca71..b86638a 100644 --- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java +++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java @@ -31,7 +31,7 @@ public class ClientOnDefaultTable extends ClientOpts { } @Parameter(names = "--table", description = "table to use") - String tableName; + public String tableName; public String getTableName() { if (tableName == null) http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java ---------------------------------------------------------------------- diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java index de85062..ea29b92 100644 --- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java +++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java @@ -25,9 +25,12 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.Properties; import java.util.Set; @@ -39,6 +42,7 @@ import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.ZooKeeperInstance; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.master.thrift.MasterGoalState; +import org.apache.accumulo.core.util.Daemon; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.server.master.Master; @@ -48,6 +52,7 @@ 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.commons.io.FileUtils; import org.apache.zookeeper.server.ZooKeeperServerMain; /** @@ -58,7 +63,7 @@ import org.apache.zookeeper.server.ZooKeeperServerMain; */ public class MiniAccumuloCluster { - public static class LogWriter extends Thread { + public static class LogWriter extends Daemon { private BufferedReader in; private BufferedWriter out; @@ -66,7 +71,6 @@ public class MiniAccumuloCluster { * @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)); @@ -108,9 +112,9 @@ public class MiniAccumuloCluster { } private boolean initialized = false; - private Process zooKeeperProcess; - private Process masterProcess; - private Process[] tabletServerProcesses; + private Process zooKeeperProcess = null; + private Process masterProcess = null; + private List tabletServerProcesses = new ArrayList(); private Set> debugPorts = new HashSet>(); @@ -140,6 +144,7 @@ public class MiniAccumuloCluster { ArrayList argList = new ArrayList(); argList.addAll(Arrays.asList(javaBin, "-cp", classpath)); + argList.add("-Djava.library.path=" + config.getLibDir()); argList.addAll(extraJvmOpts); argList.addAll(Arrays.asList("-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75", Main.class.getName(), className)); argList.addAll(Arrays.asList(args)); @@ -236,6 +241,15 @@ public class MiniAccumuloCluster { zooCfg.store(fileWriter, null); fileWriter.close(); + + File nativeMap = new File(config.getLibDir().getAbsolutePath() + "/native/map"); + nativeMap.mkdirs(); + String testRoot = new File(new File(System.getProperty("user.dir")).getParent() + "/server/src/main/c++/nativeMap").getAbsolutePath(); + for (String file : new File(testRoot).list()) { + File src = new File(testRoot, file); + if (src.isFile() && file.startsWith("libNativeMap")) + FileUtils.copyFile(src, new File(nativeMap, file)); + } } /** @@ -247,8 +261,6 @@ public class MiniAccumuloCluster { * if already started */ public void start() throws IOException, InterruptedException { - if (zooKeeperProcess != null) - throw new IllegalStateException("Already started"); if (!initialized) { @@ -265,12 +277,13 @@ public class MiniAccumuloCluster { } }); } - - zooKeeperProcess = exec(Main.class, ServerType.ZOOKEEPER, ZooKeeperServerMain.class.getName(), zooCfgFile.getAbsolutePath()); - // sleep a little bit to let zookeeper come up before calling init, seems to work better - UtilWaitThread.sleep(250); - + if (zooKeeperProcess == null) { + zooKeeperProcess = exec(Main.class, ServerType.ZOOKEEPER, ZooKeeperServerMain.class.getName(), zooCfgFile.getAbsolutePath()); + // sleep a little bit to let zookeeper come up before calling init, seems to work better + UtilWaitThread.sleep(250); + } + if (!initialized) { Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword(), "--username", "root"); int ret = initProcess.waitFor(); @@ -280,17 +293,17 @@ public class MiniAccumuloCluster { initialized = true; } - tabletServerProcesses = new Process[config.getNumTservers()]; - for (int i = 0; i < config.getNumTservers(); i++) { - tabletServerProcesses[i] = exec(TabletServer.class, ServerType.TABLET_SERVER); + for (int i = tabletServerProcesses.size(); i < config.getNumTservers(); i++) { + tabletServerProcesses.add(exec(TabletServer.class, ServerType.TABLET_SERVER)); } Process goal = exec(Main.class, SetGoalState.class.getName(), MasterGoalState.NORMAL.toString()); int ret = goal.waitFor(); if (ret != 0) { throw new RuntimeException("Could not set master goal state, process returned " + ret); } - - masterProcess = exec(Master.class, ServerType.MASTER); + if (masterProcess == null) { + masterProcess = exec(Master.class, ServerType.MASTER); + } } private List buildRemoteDebugParams(int port) { @@ -305,6 +318,53 @@ public class MiniAccumuloCluster { public Set> getDebugPorts() { return debugPorts; } + + List references(Process... procs) { + List result = new ArrayList(); + for (Process proc : procs) { + result.add(new ProcessReference(proc)); + } + return result; + } + + public Map> getProcesses() { + Map> result = new HashMap>(); + result.put(ServerType.MASTER, references(masterProcess)); + result.put(ServerType.TABLET_SERVER, references(tabletServerProcesses.toArray(new Process[0]))); + result.put(ServerType.ZOOKEEPER, references(zooKeeperProcess)); + return result; + } + + public void killProcess(ServerType type, ProcessReference proc) throws ProcessNotFoundException { + boolean found = false; + switch (type) { + case MASTER: + if (proc.equals(masterProcess)) { + masterProcess.destroy(); + masterProcess = null; + found = true; + } + break; + case TABLET_SERVER: + for (Process tserver: tabletServerProcesses) { + if (proc.equals(tserver)) { + tabletServerProcesses.remove(tserver); + found = true; + break; + } + } + break; + case ZOOKEEPER: + if (proc.equals(zooKeeperProcess)) { + zooKeeperProcess.destroy(); + zooKeeperProcess = null; + found = true; + } + break; + } + if (!found) + throw new ProcessNotFoundException(); + } /** * @return Accumulo instance name @@ -342,7 +402,7 @@ public class MiniAccumuloCluster { lw.flush(); zooKeeperProcess = null; masterProcess = null; - tabletServerProcesses = null; + tabletServerProcesses.clear(); } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessNotFoundException.java ---------------------------------------------------------------------- diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessNotFoundException.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessNotFoundException.java new file mode 100644 index 0000000..ed13760 --- /dev/null +++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessNotFoundException.java @@ -0,0 +1,23 @@ +/* + * 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; + +public class ProcessNotFoundException extends Exception { + + private static final long serialVersionUID = 1L; + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessReference.java ---------------------------------------------------------------------- diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessReference.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessReference.java new file mode 100644 index 0000000..5de99a2 --- /dev/null +++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/ProcessReference.java @@ -0,0 +1,44 @@ +/* + * 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; + +/** + * Opaque handle to a process. + */ +public class ProcessReference { + private Process process; + + ProcessReference(Process process) { + this.process = process; + } + + @Override + public String toString() { + return process.toString(); + } + + @Override + public int hashCode() { + return process.hashCode(); + } + + @Override + public boolean equals(Object obj) { + return process.equals(obj); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java ---------------------------------------------------------------------- diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java index fe2c2de..52548bf 100644 --- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java +++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java @@ -204,6 +204,7 @@ import org.apache.accumulo.trace.instrument.Trace; import org.apache.accumulo.trace.instrument.thrift.TraceWrap; import org.apache.accumulo.trace.thrift.TInfo; import org.apache.commons.collections.map.LRUMap; +import org.apache.hadoop.fs.FSError; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; @@ -1550,6 +1551,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu break; } catch (IOException ex) { log.warn("logging mutations failed, retrying"); + } catch (FSError ex) { // happens when DFS is localFS + log.warn("logging mutations failed, retrying"); } catch (Throwable t) { log.error("Unknown exception logging mutations, counts for mutations in flight not decremented!", t); throw new RuntimeException(t); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java index c6aa0b1..7d710fa 100644 --- a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java +++ b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java @@ -150,7 +150,7 @@ public class VerifyIngest { Key startKey = new Key(new Text("row_" + String.format("%010d", expectedRow))); - Scanner scanner = connector.createScanner("test_ingest", labelAuths); + Scanner scanner = connector.createScanner(opts.getTableName(), labelAuths); scanner.setBatchSize(scanOpts.scanBatchSize); scanner.setRange(new Range(startKey, endKey)); for (int j = 0; j < opts.cols; j++) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java index c522914..3fe94e1 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java @@ -17,11 +17,11 @@ package org.apache.accumulo.test.functional; import java.io.File; -import java.util.Arrays; import org.apache.accumulo.fate.zookeeper.IZooReaderWriter; import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.server.zookeeper.ZooReaderWriter; +import org.apache.commons.io.FileUtils; public class CacheTestClean { @@ -38,14 +38,9 @@ public class CacheTestClean { zoo.recursiveDelete(rootDir, NodeMissingPolicy.FAIL); } - if (!reportDir.exists()) { - reportDir.mkdir(); - } else { - File[] files = reportDir.listFiles(); - if (files.length != 0) - throw new Exception("dir " + reportDir + " is not empty: " + Arrays.asList(files)); + if (reportDir.exists()) { + FileUtils.deleteDirectory(reportDir); } - + reportDir.mkdirs(); } - } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java new file mode 100644 index 0000000..9934874 --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.accumulo.test.functional; + +import java.util.Collections; + +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.minicluster.MiniAccumuloConfig; +import org.apache.accumulo.minicluster.ProcessReference; +import org.apache.accumulo.minicluster.ServerType; +import org.apache.accumulo.server.master.Master; +import org.apache.accumulo.test.TestIngest; +import org.apache.accumulo.test.VerifyIngest; +import org.junit.Test; + +public class MasterFailoverIT extends MacTest { + + @Override + public void configure(MiniAccumuloConfig cfg) { + cfg.setSiteConfig(Collections.singletonMap(Property.INSTANCE_ZK_TIMEOUT.getKey(), "5s")); + } + + @Test(timeout=30*1000) + public void test() throws Exception { + Connector c = getConnector(); + c.tableOperations().create("test_ingest"); + TestIngest.Opts opts = new TestIngest.Opts(); + TestIngest.ingest(c, opts, BWOPTS); + for (ProcessReference master : cluster.getProcesses().get(ServerType.MASTER)) { + cluster.killProcess(ServerType.MASTER, master); + } + // start up a new one + Process p = cluster.exec(Master.class); + // talk to it + c.tableOperations().rename("test_ingest", "test_ingest2"); + try { + VerifyIngest.Opts vopts = new VerifyIngest.Opts(); + vopts.tableName = "test_ingest2"; + VerifyIngest.verifyIngest(c, vopts, SOPTS); + } finally { + p.destroy(); + } + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/main/java/org/apache/accumulo/test/functional/NativeMapTest.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapTest.java b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapTest.java index faecba0..0135557 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapTest.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapTest.java @@ -86,6 +86,7 @@ public class NativeMapTest { nmt.testBinary(); nmt.testEmpty(); nmt.testConcurrentIter(); + System.out.println("Ran to completion"); } // END JUnit methods http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java new file mode 100644 index 0000000..6ada2c2 --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.accumulo.test.functional; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class SplitRecoveryIT extends MacTest { + + @Test(timeout=10*1000) + public void test() throws Exception { + assertEquals(0, cluster.exec(SplitRecoveryTest.class).waitFor()); + } + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/test/c/fake_disk_failure.c ---------------------------------------------------------------------- diff --git a/test/src/test/c/fake_disk_failure.c b/test/src/test/c/fake_disk_failure.c new file mode 100644 index 0000000..dbb2e5e --- /dev/null +++ b/test/src/test/c/fake_disk_failure.c @@ -0,0 +1,51 @@ +/* +* 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. +*/ +#include +#include +#include +#include +#include + +static +void test_pause() { + static char trickFile[1024] = ""; + static char pid[10] = ""; + if (trickFile[0] == '\0') { + strcpy(trickFile, getenv("TRICK_FILE")); + } + + while (access(trickFile, R_OK) == 0) { + fprintf(stdout, "sleeping\n"); + fflush(stdout); + sleep(1); + } +} + +ssize_t write(int fd, const void *buf, size_t count) { + void * real_write = dlsym(RTLD_NEXT, "write"); + ssize_t (*real_write_t)(int, const void*, size_t) = real_write; + + test_pause(); + return real_write_t(fd, buf, count); +} + +ssize_t read(int fd, void *buf, size_t count) { + void * real_read = dlsym(RTLD_NEXT, "read"); + ssize_t (*real_read_t)(int, void*, size_t) = real_read; + test_pause(); + return real_read_t(fd, buf, count); +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java ---------------------------------------------------------------------- diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java new file mode 100644 index 0000000..597722b --- /dev/null +++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java @@ -0,0 +1,865 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.accumulo.test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.PrintWriter; +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.util.Map.Entry; + +import jline.console.ConsoleReader; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.file.FileOperations; +import org.apache.accumulo.core.file.FileSKVWriter; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.core.util.shell.Shell; +import org.apache.accumulo.minicluster.MiniAccumuloCluster; +import org.apache.accumulo.minicluster.MiniAccumuloConfig; +import org.apache.accumulo.server.trace.TraceServer; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.tools.DistCp; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class ShellServerIT { + public static class TestOutputStream extends OutputStream { + StringBuilder sb = new StringBuilder(); + + @Override + public void write(int b) throws IOException { + sb.append((char) (0xff & b)); + } + + public String get() { + return sb.toString(); + } + + public void clear() { + sb.setLength(0); + } + } + + public static class StringInputStream extends InputStream { + private String source = ""; + private int offset = 0; + + @Override + public int read() throws IOException { + if (offset == source.length()) + return '\n'; + else + return source.charAt(offset++); + } + + public void set(String other) { + source = other; + offset = 0; + } + } + + private static String secret = "superSecret"; + public static TemporaryFolder folder = new TemporaryFolder(); + public static MiniAccumuloCluster cluster; + public static TestOutputStream output; + public static StringInputStream input; + public static Shell shell; + private static Process traceProcess; + + static String exec(String cmd) throws IOException { + output.clear(); + shell.execCommand(cmd, true, true); + return output.get(); + } + + static String exec(String cmd, boolean expectGoodExit) throws IOException { + String result = exec(cmd); + if (expectGoodExit) + assertGoodExit("", true); + else + assertBadExit("", true); + return result; + } + + static String exec(String cmd, boolean expectGoodExit, String expectString) throws IOException { + return exec(cmd, expectGoodExit, expectString, true); + } + + static String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent) throws IOException { + String result = exec(cmd); + if (expectGoodExit) + assertGoodExit(expectString, stringPresent); + else + assertBadExit(expectString, stringPresent); + return result; + } + + static void assertGoodExit(String s, boolean stringPresent) { + Shell.log.info(output.get()); + assertEquals(0, shell.getExitCode()); + + if (s.length() > 0) + assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s)); + } + + static void assertBadExit(String s, boolean stringPresent) { + Shell.log.debug(output.get()); + assertTrue(shell.getExitCode() > 0); + if (s.length() > 0) + assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s)); + shell.resetExitCode(); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + folder.create(); + MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret); + cluster = new MiniAccumuloCluster(cfg); + cluster.start(); + + // history file is updated in $HOME + System.setProperty("HOME", folder.getRoot().getAbsolutePath()); + + // start the shell + output = new TestOutputStream(); + input = new StringInputStream(); + shell = new Shell(new ConsoleReader(input, output)); + shell.setLogErrorsToConsole(); + shell.config("-u", "root", "-p", secret, "-z", cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers()); + exec("quit", true); + shell.start(); + shell.setExit(false); + + // use reflection to call this method so it does not need to be made public + Method method = cluster.getClass().getDeclaredMethod("exec", Class.class, String[].class); + method.setAccessible(true); + traceProcess = (Process) method.invoke(cluster, TraceServer.class, new String[0]); + + // give the tracer some time to start + UtilWaitThread.sleep(1000); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + cluster.stop(); + traceProcess.destroy(); + folder.delete(); + } + + @After + public void tearDown() throws Exception { + Connector c = cluster.getConnector("root", secret); + for (String table : c.tableOperations().list()) { + if (!table.equals(MetadataTable.NAME) && !table.equals(RootTable.NAME)) + c.tableOperations().delete(table); + } + } + + @Test(timeout = 30*1000) + public void exporttableImporttable() throws Exception { + // exporttable / importtable + exec("createtable t -evc", true); + make10(); + exec("addsplits row5", true); + exec("config -t t -s table.split.threshold=345M", true); + exec("offline t", true); + String export = "file://" + folder.newFolder().toString(); + exec("exporttable -t t " + export, true); + DistCp cp = newDistCp(); + String import_ = "file://" + folder.newFolder().toString(); + cp.run(new String[] {"-f", export + "/distcp.txt", import_}); + exec("importtable t2 " + import_, true); + exec("config -t t2 -np", true, "345M", true); + exec("getsplits -t t2", true, "row5", true); + exec("constraint --list -t t2", true, "VisibilityConstraint=2", true); + exec("onlinetable t", true); + exec("deletetable -f t", true); + exec("deletetable -f t2", true); + } + + private DistCp newDistCp() { + try { + @SuppressWarnings("unchecked") + Constructor[] constructors = (Constructor[]) DistCp.class.getConstructors(); + for (Constructor constructor : constructors) { + Class[] parameterTypes = constructor.getParameterTypes(); + if (parameterTypes.length > 0 && parameterTypes[0].equals(Configuration.class)) { + if (parameterTypes.length == 1) { + return constructor.newInstance(new Configuration()); + } else if (parameterTypes.length == 2) { + return constructor.newInstance(new Configuration(), null); + } + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Unexpected constructors for DistCp"); + } + + @Test(timeout = 30 * 1000) + public void setscaniterDeletescaniter() throws Exception { + // setscaniter, deletescaniter + exec("createtable t"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + input.set("true\n\n\nSTRING"); + exec("setscaniter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true); + exec("scan", true, "3", true); + exec("deletescaniter -n name", true); + exec("scan", true, "1", true); + exec("deletetable -f t"); + + } + + @Test(timeout = 30 * 1000) + public void execfile() throws Exception { + // execfile + File file = folder.newFile(); + PrintWriter writer = new PrintWriter(file.getAbsolutePath()); + writer.println("about"); + writer.close(); + exec("execfile " + file.getAbsolutePath(), true, Constants.VERSION, true); + + } + + @Test(timeout = 30 * 1000) + public void egrep() throws Exception { + // egrep + exec("createtable t"); + make10(); + String lines = exec("egrep row[123]", true); + assertTrue(lines.split("\n").length - 1 == 3); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void du() throws Exception { + // du + exec("createtable t"); + make10(); + exec("flush -t t -w"); + exec("du t", true, " [t]", true); + output.clear(); + shell.execCommand("du -h", false, false); + String o = output.get(); + assertTrue(o.matches(".*26[0-9]\\s\\[t\\]\\n")); // for some reason, there's 1-2 bytes of fluctuation + exec("deletetable -f t"); + } + + @Test(timeout = 1000) + public void debug() throws Exception { + exec("debug", true, "off", true); + exec("debug on", true); + exec("debug", true, "on", true); + exec("debug off", true); + exec("debug", true, "off", true); + exec("debug debug", false); + exec("debug debug debug", false); + } + + @Test(timeout = 30 * 1000) + public void user() throws Exception { + // createuser, deleteuser, user, users, droptable, grant, revoke + input.set("secret\nsecret\n"); + exec("createuser xyzzy", true); + exec("users", true, "xyzzy", true); + String perms = exec("userpermissions -u xyzzy", true); + assertTrue(perms.contains("Table permissions (!METADATA): Table.READ")); + exec("grant -u xyzzy -s System.CREATE_TABLE", true); + perms = exec("userpermissions -u xyzzy", true); + assertTrue(perms.contains("")); + exec("grant -u root -t !METADATA Table.WRITE", true); + exec("grant -u root -t !METADATA Table.GOOFY", false); + exec("grant -u root -s foo", false); + exec("grant -u xyzzy -t !METADATA foo", false); + input.set("secret\nsecret\n"); + exec("user xyzzy", true); + exec("createtable t", true, "xyzzy@", true); + exec("insert row1 cf cq 1", true); + exec("scan", true, "row1", true); + exec("droptable -f t", true); + exec("deleteuser xyzzy", false, "delete yourself", true); + input.set(secret + "\n" + secret + "\n"); + exec("user root", true); + exec("revoke -u xyzzy -s System.CREATE_TABLE", true); + exec("revoke -u xyzzy -s System.GOOFY", false); + exec("revoke -u xyzzy -s foo", false); + exec("revoke -u xyzzy -t !METADATA Table.WRITE", true); + exec("revoke -u xyzzy -t !METADATA Table.GOOFY", false); + exec("revoke -u xyzzy -t !METADATA foo", false); + exec("deleteuser xyzzy", true); + exec("users", true, "xyzzy", false); + } + + @Test(timeout = 30 * 1000) + public void iter() throws Exception { + // setshelliter, listshelliter, deleteshelliter + exec("createtable t"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + input.set("true\n\n\nSTRING\n"); + exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n name", true); + exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n name", false); + exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n other", false); + input.set("true\n\n\nSTRING\n"); + exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n xyzzy", true); + exec("scan -pn sum", true, "3", true); + exec("listshelliter", true, "Iterator name", true); + exec("listshelliter", true, "Iterator xyzzy", true); + exec("listshelliter", true, "Profile : sum", true); + exec("deleteshelliter -pn sum -n name", true); + exec("listshelliter", true, "Iterator name", false); + exec("listshelliter", true, "Iterator xyzzy", true); + exec("deleteshelliter -pn sum -a", true); + exec("listshelliter", true, "Iterator xyzzy", false); + exec("listshelliter", true, "Profile : sum", false); + exec("deletetable -f t"); + // list iter + exec("createtable t"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + exec("insert a cf cq 1"); + input.set("true\n\n\nSTRING\n"); + exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true); + exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n name", false); + exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n other", false); + input.set("true\n\n\nSTRING\n"); + exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n xyzzy", true); + exec("scan", true, "3", true); + exec("listiter -scan", true, "Iterator name", true); + exec("listiter -scan", true, "Iterator xyzzy", true); + exec("listiter -minc", true, "Iterator name", false); + exec("listiter -minc", true, "Iterator xyzzy", false); + exec("deleteiter -scan -n name", true); + exec("listiter -scan", true, "Iterator name", false); + exec("listiter -scan", true, "Iterator xyzzy", true); + exec("deletetable -f t"); + + } + + @Test(timeout = 30 * 1000) + public void notable() throws Exception { + // notable + exec("createtable xyzzy", true); + exec("scan", true, " xyzzy>", true); + assertTrue(output.get().contains(" xyzzy>")); + exec("notable", true); + exec("scan", false, "Not in a table context.", true); + assertFalse(output.get().contains(" xyzzy>")); + exec("deletetable -f xyzzy"); + } + + @Test(timeout = 30 * 1000) + public void sleep() throws Exception { + // sleep + long now = System.currentTimeMillis(); + exec("sleep 0.2", true); + long diff = System.currentTimeMillis() - now; + assertTrue(diff >= 200); + assertTrue(diff < 400); + } + + @Test(timeout = 30 * 1000) + public void addauths() throws Exception { + // addauths + exec("createtable xyzzy -evc"); + exec("insert a b c d -l foo", false, "does not have authorization", true); + exec("addauths -s foo,bar", true); + exec("getauths", true, "foo", true); + exec("getauths", true, "bar", true); + exec("insert a b c d -l foo"); + exec("scan", true, "[foo]"); + exec("scan -s bar", true, "[foo]", false); + exec("deletetable -f xyzzy"); + } + + @Test(timeout = 30 * 1000) + public void byeQuitExit() throws Exception { + // bye, quit, exit + for (String cmd : "bye quit exit".split(" ")) { + assertFalse(shell.getExit()); + exec(cmd); + assertTrue(shell.getExit()); + shell.setExit(false); + } + } + + @Test(timeout = 30 * 1000) + public void classpath() throws Exception { + // classpath + exec("classpath", true, "Level 2: Java Classloader (loads everything defined by java classpath) URL classpath items are", true); + } + + @Test(timeout = 30 * 1000) + public void clearCls() throws Exception { + // clear/cls + if (shell.getReader().getTerminal().isAnsiSupported()) { + exec("cls", true, "[1;1H"); + exec("clear", true, "[2J"); + } else { + exec("cls", false, "does not support"); + exec("clear", false, "does not support"); + } + } + + @Test(timeout = 30 * 1000) + public void clonetable() throws Exception { + // clonetable + exec("createtable orig -evc"); + exec("config -t orig -s table.split.threshold=123M", true); + exec("addsplits -t orig a b c", true); + exec("insert a b c value"); + exec("scan", true, "value", true); + exec("clonetable orig clone"); + // verify constraint, config, and splits were cloned + exec("constraint --list -t clone", true, "VisibilityConstraint=2", true); + exec("config -t clone -np", true, "123M", true); + exec("getsplits -t clone", true, "a\nb\nc\n"); + // compact + exec("createtable c"); + // make two files + exec("insert a b c d"); + exec("flush -w"); + exec("insert x y z v"); + exec("flush -w"); + int oldCount = countFiles(); + // merge two files into one + exec("compact -t c -w"); + assertTrue(countFiles() < oldCount); + exec("addsplits -t c f"); + // make two more files: + exec("insert m 1 2 3"); + exec("flush -w"); + exec("insert n 1 2 3"); + exec("flush -w"); + oldCount = countFiles(); + // at this point there are 3 files in the default tablet + // compact some data: + exec("compact -b g -e z -w"); + assertTrue(countFiles() == oldCount - 2); + exec("compact -w"); + assertTrue(countFiles() == oldCount - 2); + exec("merge --all -t c"); + exec("compact -w"); + assertTrue(countFiles() == oldCount - 3); + exec("deletetable -f orig"); + exec("deletetable -f clone"); + exec("deletetable -f c"); + } + + @Test(timeout = 30 * 1000) + public void constraint() throws Exception { + // constraint + exec("constraint -l -t !METADATA", true, "MetadataConstraints=1", true); + exec("createtable c -evc"); + exec("constraint -l -t c", true, "VisibilityConstraint=2", true); + exec("constraint -t c -d 2", true, "Removed constraint 2 from table c"); + exec("constraint -l -t c", true, "VisibilityConstraint=2", false); + exec("deletetable -f c"); + } + + @Test(timeout = 30 * 1000) + public void deletemany() throws Exception { + // deletemany + exec("createtable t"); + make10(); + assertEquals(10, countkeys("t")); + exec("deletemany -f -b row8"); + assertEquals(8, countkeys("t")); + exec("scan -t t -np", true, "row8", false); + make10(); + exec("deletemany -f -b row4 -e row5"); + assertEquals(8, countkeys("t")); + make10(); + exec("deletemany -f -c cf:col4,cf:col5"); + assertEquals(8, countkeys("t")); + make10(); + exec("deletemany -f -r row3"); + assertEquals(9, countkeys("t")); + make10(); + exec("deletemany -f -r row3"); + assertEquals(9, countkeys("t")); + make10(); + exec("deletemany -f -b row3 -be -e row5 -ee"); + assertEquals(9, countkeys("t")); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void deleterows() throws Exception { + // deleterows + int base = countFiles(); + exec("createtable t"); + exec("addsplits row5 row7"); + make10(); + exec("flush -w -t t"); + assertTrue(base + 3 == countFiles()); + exec("deleterows -t t -b row5 -e row7", true); + assertTrue(base + 2 == countFiles()); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void groups() throws Exception { + exec("createtable t"); + exec("setgroups -t t alpha=a,b,c num=3,2,1"); + exec("getgroups -t t", true, "alpha=a,b,c", true); + exec("getgroups -t t", true, "num=1,2,3", true); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void grep() throws Exception { + exec("createtable t", true); + make10(); + exec("grep row[123]", true, "row1", false); + exec("grep row5", true, "row5", true); + exec("deletetable -f t", true); + } + + @Test + // (timeout = 30 * 1000) + public void help() throws Exception { + exec("help -np", true, "Help Commands", true); + exec("?", true, "Help Commands", true); + for (String c : ("bye exit quit " + "about help info ? " + "deleteiter deletescaniter listiter setiter setscaniter " + + "grant revoke systempermissions tablepermissions userpermissions " + "execfile history " + "authenticate cls clear notable sleep table user whoami " + + "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables " + + "addsplits compact constraint flush getgropus getsplits merge setgroups " + "addauths createuser deleteuser dropuser getauths passwd setauths users " + + "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan").split(" ")) { + exec("help " + c, true); + } + } + + // @Test(timeout = 30 * 1000) + public void history() throws Exception { + exec("history -c", true); + exec("createtable unusualstring"); + exec("deletetable -f unusualstring"); + exec("history", true, "unusualstring", true); + exec("history", true, "history", true); + } + + @Test(timeout = 30 * 1000) + public void importDirectory() throws Exception { + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.get(conf); + File importDir = folder.newFolder("import"); + String even = new File(importDir, "even.rf").toString(); + String odd = new File(importDir, "odd.rf").toString(); + File errorsDir = folder.newFolder("errors"); + fs.mkdirs(new Path(errorsDir.toString())); + AccumuloConfiguration aconf = AccumuloConfiguration.getDefaultConfiguration(); + FileSKVWriter evenWriter = FileOperations.getInstance().openWriter(even, fs, conf, aconf); + evenWriter.startDefaultLocalityGroup(); + FileSKVWriter oddWriter = FileOperations.getInstance().openWriter(odd, fs, conf, aconf); + oddWriter.startDefaultLocalityGroup(); + long ts = System.currentTimeMillis(); + Text cf = new Text("cf"); + Text cq = new Text("cq"); + Value value = new Value("value".getBytes()); + for (int i = 0; i < 100; i += 2) { + Key key = new Key(new Text(String.format("%8d", i)), cf, cq, ts); + evenWriter.append(key, value); + key = new Key(new Text(String.format("%8d", i + 1)), cf, cq, ts); + oddWriter.append(key, value); + } + evenWriter.close(); + oddWriter.close(); + assertEquals(0, shell.getExitCode()); + exec("createtable t", true); + exec("importdirectory " + importDir + " " + errorsDir + " true", true); + exec("scan -r 00000000", true, "00000000", true); + exec("scan -r 00000099", true, "00000099", true); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void info() throws Exception { + exec("info", true, Constants.VERSION, true); + } + + @Test(timeout = 30 * 1000) + public void interpreter() throws Exception { + exec("createtable t", true); + exec("interpreter -l", true, "HexScan", false); + exec("insert \\x02 cf cq value", true); + exec("scan -b 02", true, "value", false); + exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true); + exec("interpreter -l", true, "HexScan", true); + exec("scan -b 02", true, "value", true); + exec("deletetable -f t", true); + } + + @Test(timeout = 30 * 1000) + public void listcompactions() throws Exception { + exec("createtable t", true); + exec("config -t t -s table.iterator.minc.slow=30,org.apache.accumulo.test.functional.SlowIterator", true); + exec("config -t t -s table.iterator.minc.slow.opt.sleepTime=100", true); + exec("insert a cf cq value", true); + exec("insert b cf cq value", true); + exec("insert c cf cq value", true); + exec("insert d cf cq value", true); + exec("flush -t t", true); + exec("sleep 0.2", true); + exec("listcompactions", true, "default_tablet"); + String[] lines = output.get().split("\n"); + String last = lines[lines.length - 1]; + String[] parts = last.split("\\|"); + assertEquals(12, parts.length); + exec("deletetable -f t", true); + } + + @Test(timeout = 30 * 1000) + public void maxrow() throws Exception { + exec("createtable t", true); + exec("insert a cf cq value", true); + exec("insert b cf cq value", true); + exec("insert ccc cf cq value", true); + exec("insert zzz cf cq value", true); + exec("maxrow", true, "zzz", true); + exec("delete zzz cf cq", true); + exec("maxrow", true, "ccc", true); + exec("deletetable -f t", true); + } + + @Test(timeout = 30 * 1000) + public void merge() throws Exception { + exec("createtable t"); + exec("addsplits a m z"); + exec("getsplits", true, "z", true); + exec("merge --all", true); + exec("getsplits", true, "z", false); + exec("deletetable -f t"); + exec("getsplits -t !METADATA", true); + assertEquals(2, output.get().split("\n").length); + exec("getsplits -t !!ROOT", true); + assertEquals(1, output.get().split("\n").length); + exec("merge --all -t !METADATA"); + exec("getsplits -t !METADATA", true); + assertEquals(1, output.get().split("\n").length); + } + + @Test(timeout = 30 * 1000) + public void ping() throws Exception { + for (int i = 0; i < 10; i++) { + exec("ping", true, "OK", true); + // wait for both tservers to start up + if (output.get().split("\n").length == 3) + break; + UtilWaitThread.sleep(1000); + + } + assertEquals(3, output.get().split("\n").length); + } + + @Test(timeout = 30 * 1000) + public void renametable() throws Exception { + exec("createtable aaaa"); + exec("insert this is a value"); + exec("renametable aaaa xyzzy"); + exec("tables", true, "xyzzy", true); + exec("tables", true, "aaaa", false); + exec("scan -t xyzzy", true, "value", true); + exec("deletetable -f xyzzy", true); + } + + @Test(timeout = 30 * 1000) + public void systempermission() throws Exception { + exec("systempermissions"); + assertEquals(8, output.get().split("\n").length - 1); + exec("tablepermissions", true); + assertEquals(6, output.get().split("\n").length - 1); + } + + @Test(timeout = 30 * 1000) + public void listscans() throws Exception { + exec("createtable t", true); + exec("config -t t -s table.iterator.scan.slow=30,org.apache.accumulo.test.functional.SlowIterator", true); + exec("config -t t -s table.iterator.scan.slow.opt.sleepTime=100", true); + exec("insert a cf cq value", true); + exec("insert b cf cq value", true); + exec("insert c cf cq value", true); + exec("insert d cf cq value", true); + Thread thread = new Thread() { + @Override + public void run() { + try { + Connector connector = cluster.getConnector("root", secret); + Scanner s = connector.createScanner("t", Authorizations.EMPTY); + for (@SuppressWarnings("unused") + Entry kv : s) + ; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + }; + thread.start(); + exec("sleep 0.1", true); + String scans = exec("listscans", true); + String lines[] = scans.split("\n"); + String last = lines[lines.length - 1]; + assertTrue(last.contains("RUNNING")); + String parts[] = last.split("\\|"); + assertEquals(13, parts.length); + thread.join(); + exec("deletetable -f t", true); + } + + @Test(timeout = 30 * 1000) + public void testPertableClasspath() throws Exception { + File fooFilterJar = File.createTempFile("FooFilter", ".jar"); + FileUtils.copyURLToFile(this.getClass().getResource("/FooFilter.jar"), fooFilterJar); + fooFilterJar.deleteOnExit(); + + File fooConstraintJar = File.createTempFile("FooConstraint", ".jar"); + FileUtils.copyURLToFile(this.getClass().getResource("/FooConstraint.jar"), fooConstraintJar); + fooConstraintJar.deleteOnExit(); + + exec( + "config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1=" + fooFilterJar.toURI().toString() + "," + fooConstraintJar.toURI().toString(), + true); + + exec("createtable ptc", true); + exec("config -t ptc -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true); + + UtilWaitThread.sleep(200); + + exec("setiter -scan -class org.apache.accumulo.test.FooFilter -p 10 -n foo", true); + + exec("insert foo f q v", true); + + UtilWaitThread.sleep(100); + + exec("scan -np", true, "foo", false); + + exec("constraint -a FooConstraint", true); + + exec("offline ptc"); + UtilWaitThread.sleep(500); + exec("online ptc"); + + exec("table ptc", true); + exec("insert foo f q v", false); + exec("insert ok foo q v", true); + + exec("deletetable ptc", true); + exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1"); + + } + + @Test(timeout = 30 * 1000) + public void trace() throws Exception { + exec("trace on", true); + exec("createtable t", true); + exec("insert a b c value", true); + exec("scan -np", true, "value", true); + exec("deletetable -f t"); + exec("sleep 1"); + String trace = exec("trace off"); + System.out.println(trace); + assertTrue(trace.contains("sendMutations")); + assertTrue(trace.contains("startScan")); + assertTrue(trace.contains("DeleteTable")); + } + + @Test(timeout=30 * 1000) + public void badLogin() throws Exception { + input.set(secret + "\n"); + String err = exec("user NoSuchUser", false); + assertTrue(err.contains("BAD_CREDENTIALS for user NoSuchUser")); + } + + private int countkeys(String table) throws IOException { + exec("scan -np -t " + table); + return output.get().split("\n").length - 1; + } + + @Test(timeout = 30 * 1000) + public void scans() throws Exception { + exec("createtable t"); + make10(); + String result = exec("scan -np -b row1 -e row1"); + assertEquals(2, result.split("\n").length); + result = exec("scan -np -b row3 -e row5"); + assertEquals(4, result.split("\n").length); + result = exec("scan -np -r row3"); + assertEquals(2, result.split("\n").length); + result = exec("scan -np -b row:"); + assertEquals(1, result.split("\n").length); + result = exec("scan -np -b row"); + assertEquals(11, result.split("\n").length); + result = exec("scan -np -e row:"); + assertEquals(11, result.split("\n").length); + exec("deletetable -f t"); + } + + @Test(timeout = 30 * 1000) + public void whoami() throws Exception { + assertTrue(exec("whoami", true).contains("root")); + input.set("secret\nsecret\n"); + exec("createuser test_user"); + exec("setauths -u test_user -s 12,3,4"); + String auths = exec("getauths -u test_user"); + assertTrue(auths.contains("3") && auths.contains("12") && auths.contains("4")); + input.set("secret\n"); + exec("user test_user", true); + assertTrue(exec("whoami", true).contains("test_user")); + input.set(secret + "\n"); + exec("user root", true); + } + + private void make10() throws IOException { + for (int i = 0; i < 10; i++) { + exec(String.format("insert row%d cf col%d value", i, i)); + } + } + + private int countFiles() throws IOException { + exec("scan -t !METADATA -np -c file"); + return output.get().split("\n").length - 1; + } + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a1075a4/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java ---------------------------------------------------------------------- diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java b/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java deleted file mode 100644 index f2f4ebe..0000000 --- a/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java +++ /dev/null @@ -1,805 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.accumulo.test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.PrintWriter; -import java.lang.reflect.Constructor; -import java.lang.reflect.Method; -import java.util.Map.Entry; - -import jline.console.ConsoleReader; - -import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.Connector; -import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.file.FileOperations; -import org.apache.accumulo.core.file.FileSKVWriter; -import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.util.UtilWaitThread; -import org.apache.accumulo.core.util.shell.Shell; -import org.apache.accumulo.minicluster.MiniAccumuloCluster; -import org.apache.accumulo.minicluster.MiniAccumuloConfig; -import org.apache.accumulo.server.trace.TraceServer; -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.tools.DistCp; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class ShellServerTest { - public static class TestOutputStream extends OutputStream { - StringBuilder sb = new StringBuilder(); - - @Override - public void write(int b) throws IOException { - sb.append((char) (0xff & b)); - } - - public String get() { - return sb.toString(); - } - - public void clear() { - sb.setLength(0); - } - } - - public static class StringInputStream extends InputStream { - private String source = ""; - private int offset = 0; - - @Override - public int read() throws IOException { - if (offset == source.length()) - return '\n'; - else - return source.charAt(offset++); - } - - public void set(String other) { - source = other; - offset = 0; - } - } - - private static String secret = "superSecret"; - public static TemporaryFolder folder = new TemporaryFolder(); - public static MiniAccumuloCluster cluster; - public static TestOutputStream output; - public static StringInputStream input; - public static Shell shell; - private static Process traceProcess; - - static String exec(String cmd) throws IOException { - output.clear(); - shell.execCommand(cmd, true, true); - return output.get(); - } - - static String exec(String cmd, boolean expectGoodExit) throws IOException { - String result = exec(cmd); - if (expectGoodExit) - assertGoodExit("", true); - else - assertBadExit("", true); - return result; - } - - static String exec(String cmd, boolean expectGoodExit, String expectString) throws IOException { - return exec(cmd, expectGoodExit, expectString, true); - } - - static String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent) throws IOException { - String result = exec(cmd); - if (expectGoodExit) - assertGoodExit(expectString, stringPresent); - else - assertBadExit(expectString, stringPresent); - return result; - } - - static void assertGoodExit(String s, boolean stringPresent) { - Shell.log.info(output.get()); - assertEquals(0, shell.getExitCode()); - - if (s.length() > 0) - assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s)); - } - - static void assertBadExit(String s, boolean stringPresent) { - Shell.log.debug(output.get()); - assertTrue(shell.getExitCode() > 0); - if (s.length() > 0) - assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s)); - shell.resetExitCode(); - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - folder.create(); - MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret); - cluster = new MiniAccumuloCluster(cfg); - cluster.start(); - - System.setProperty("HOME", folder.getRoot().getAbsolutePath()); - - // start the shell - output = new TestOutputStream(); - input = new StringInputStream(); - shell = new Shell(new ConsoleReader(input, output)); - shell.setLogErrorsToConsole(); - shell.config("-u", "root", "-p", secret, "-z", cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers()); - exec("quit", true); - shell.start(); - shell.setExit(false); - - // use reflection to call this method so it does not need to be made public - Method method = cluster.getClass().getDeclaredMethod("exec", Class.class, String[].class); - method.setAccessible(true); - traceProcess = (Process) method.invoke(cluster, TraceServer.class, new String[0]); - - // give the tracer some time to start - UtilWaitThread.sleep(1000); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - cluster.stop(); - traceProcess.destroy(); - folder.delete(); - } - - @Test(timeout = 30000) - public void exporttableImporttable() throws Exception { - // exporttable / importtable - exec("createtable t -evc", true); - make10(); - exec("addsplits row5", true); - exec("config -t t -s table.split.threshold=345M", true); - exec("offline t", true); - String export = "file://" + folder.newFolder().toString(); - exec("exporttable -t t " + export, true); - DistCp cp = newDistCp(); - String import_ = "file://" + folder.newFolder().toString(); - cp.run(new String[] {"-f", export + "/distcp.txt", import_}); - exec("importtable t2 " + import_, true); - exec("config -t t2 -np", true, "345M", true); - exec("getsplits -t t2", true, "row5", true); - exec("constraint --list -t t2", true, "VisibilityConstraint=2", true); - exec("onlinetable t", true); - exec("deletetable -f t", true); - exec("deletetable -f t2", true); - } - - private DistCp newDistCp() { - try { - @SuppressWarnings("unchecked") - Constructor[] constructors = (Constructor[]) DistCp.class.getConstructors(); - for (Constructor constructor : constructors) { - Class[] parameterTypes = constructor.getParameterTypes(); - if (parameterTypes.length > 0 && parameterTypes[0].equals(Configuration.class)) { - if (parameterTypes.length == 1) { - return constructor.newInstance(new Configuration()); - } else if (parameterTypes.length == 2) { - return constructor.newInstance(new Configuration(), null); - } - } - } - } catch (Exception e) { - throw new RuntimeException(e); - } - throw new RuntimeException("Unexpected constructors for DistCp"); - } - - @Test(timeout = 30000) - public void setscaniterDeletescaniter() throws Exception { - // setscaniter, deletescaniter - exec("createtable t"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - input.set("true\n\n\nSTRING"); - exec("setscaniter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true); - exec("scan", true, "3", true); - exec("deletescaniter -n name", true); - exec("scan", true, "1", true); - exec("deletetable -f t"); - - } - - @Test(timeout = 30000) - public void execfile() throws Exception { - // execfile - File file = folder.newFile(); - PrintWriter writer = new PrintWriter(file.getAbsolutePath()); - writer.println("about"); - writer.close(); - exec("execfile " + file.getAbsolutePath(), true, Constants.VERSION, true); - - } - - @Test(timeout = 30000) - public void egrep() throws Exception { - // egrep - exec("createtable t"); - make10(); - String lines = exec("egrep row[123]", true); - assertTrue(lines.split("\n").length - 1 == 3); - exec("deletetable -f t"); - } - - @Test(timeout = 30000) - public void du() throws Exception { - // du - exec("createtable t"); - make10(); - exec("flush -t t -w"); - exec("du t", true, " [t]", true); - output.clear(); - shell.execCommand("du -h", false, false); - String o = output.get(); - assertTrue(o.matches(".*26[0-9]\\s\\[t\\]\\n")); // for some reason, there's 1-2 bytes of fluctuation - exec("deletetable -f t"); - } - - @Test(timeout = 1000) - public void debug() throws Exception { - exec("debug", true, "off", true); - exec("debug on", true); - exec("debug", true, "on", true); - exec("debug off", true); - exec("debug", true, "off", true); - exec("debug debug", false); - exec("debug debug debug", false); - } - - @Test(timeout = 30000) - public void user() throws Exception { - // createuser, deleteuser, user, users, droptable, grant, revoke - input.set("secret\nsecret\n"); - exec("createuser xyzzy", true); - exec("users", true, "xyzzy", true); - String perms = exec("userpermissions -u xyzzy", true); - assertTrue(perms.contains("Table permissions (!METADATA): Table.READ")); - exec("grant -u xyzzy -s System.CREATE_TABLE", true); - perms = exec("userpermissions -u xyzzy", true); - assertTrue(perms.contains("")); - exec("grant -u root -t !METADATA Table.WRITE", true); - exec("grant -u root -t !METADATA Table.GOOFY", false); - exec("grant -u root -s foo", false); - exec("grant -u xyzzy -t !METADATA foo", false); - input.set("secret\nsecret\n"); - exec("user xyzzy", true); - exec("createtable t", true, "xyzzy@", true); - exec("insert row1 cf cq 1", true); - exec("scan", true, "row1", true); - exec("droptable -f t", true); - exec("deleteuser xyzzy", false, "delete yourself", true); - input.set(secret + "\n" + secret + "\n"); - exec("user root", true); - exec("revoke -u xyzzy -s System.CREATE_TABLE", true); - exec("revoke -u xyzzy -s System.GOOFY", false); - exec("revoke -u xyzzy -s foo", false); - exec("revoke -u xyzzy -t !METADATA Table.WRITE", true); - exec("revoke -u xyzzy -t !METADATA Table.GOOFY", false); - exec("revoke -u xyzzy -t !METADATA foo", false); - exec("deleteuser xyzzy", true); - exec("users", true, "xyzzy", false); - } - - @Test(timeout = 30000) - public void iter() throws Exception { - // setshelliter, listshelliter, deleteshelliter - exec("createtable t"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - input.set("true\n\n\nSTRING\n"); - exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n name", true); - input.set("true\n\n\nSTRING\n"); - exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n xyzzy", true); - exec("scan -pn sum", true, "3", true); - exec("listshelliter", true, "Iterator name", true); - exec("listshelliter", true, "Iterator xyzzy", true); - exec("listshelliter", true, "Profile : sum", true); - exec("deleteshelliter -pn sum -n name", true); - exec("listshelliter", true, "Iterator name", false); - exec("listshelliter", true, "Iterator xyzzy", true); - exec("deleteshelliter -pn sum -a", true); - exec("listshelliter", true, "Iterator xyzzy", false); - exec("listshelliter", true, "Profile : sum", false); - exec("deletetable -f t"); - // list iter - exec("createtable t"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - exec("insert a cf cq 1"); - input.set("true\n\n\nSTRING\n"); - exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true); - input.set("true\n\n\nSTRING\n"); - exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n xyzzy", true); - exec("scan", true, "3", true); - exec("listiter -scan", true, "Iterator name", true); - exec("listiter -scan", true, "Iterator xyzzy", true); - exec("listiter -minc", true, "Iterator name", false); - exec("listiter -minc", true, "Iterator xyzzy", false); - exec("deleteiter -scan -n name", true); - exec("listiter -scan", true, "Iterator name", false); - exec("listiter -scan", true, "Iterator xyzzy", true); - exec("deletetable -f t"); - - } - - @Test(timeout = 30000) - public void notable() throws Exception { - // notable - exec("createtable xyzzy", true); - exec("scan", true, " xyzzy>", true); - assertTrue(output.get().contains(" xyzzy>")); - exec("notable", true); - exec("scan", false, "Not in a table context.", true); - assertFalse(output.get().contains(" xyzzy>")); - exec("deletetable -f xyzzy"); - } - - @Test(timeout = 30000) - public void sleep() throws Exception { - // sleep - long now = System.currentTimeMillis(); - exec("sleep 0.2", true); - long diff = System.currentTimeMillis() - now; - assertTrue(diff >= 200); - assertTrue(diff < 400); - } - - @Test(timeout = 30000) - public void addauths() throws Exception { - // addauths - exec("createtable xyzzy -evc"); - exec("insert a b c d -l foo", false, "does not have authorization", true); - exec("addauths -s foo,bar", true); - exec("getauths", true, "foo", true); - exec("getauths", true, "bar", true); - exec("insert a b c d -l foo"); - exec("scan", true, "[foo]"); - exec("scan -s bar", true, "[foo]", false); - exec("deletetable -f xyzzy"); - } - - @Test(timeout = 30000) - public void byeQuitExit() throws Exception { - // bye, quit, exit - for (String cmd : "bye quit exit".split(" ")) { - assertFalse(shell.getExit()); - exec(cmd); - assertTrue(shell.getExit()); - shell.setExit(false); - } - } - - @Test(timeout = 30000) - public void classpath() throws Exception { - // classpath - exec("classpath", true, "Level 2: Java Classloader (loads everything defined by java classpath) URL classpath items are", true); - } - - @Test(timeout = 30000) - public void clearCls() throws Exception { - // clear/cls - if (shell.getReader().getTerminal().isAnsiSupported()) { - exec("cls", true, "[1;1H"); - exec("clear", true, "[2J"); - } else { - exec("cls", false, "does not support"); - exec("clear", false, "does not support"); - } - } - - @Test(timeout = 30000) - public void clonetable() throws Exception { - // clonetable - exec("createtable orig -evc"); - exec("config -t orig -s table.split.threshold=123M", true); - exec("addsplits -t orig a b c", true); - exec("insert a b c value"); - exec("scan", true, "value", true); - exec("clonetable orig clone"); - // verify constraint, config, and splits were cloned - exec("constraint --list -t clone", true, "VisibilityConstraint=2", true); - exec("config -t clone -np", true, "123M", true); - exec("getsplits -t clone", true, "a\nb\nc\n"); - // compact - exec("createtable c"); - // make two files - exec("insert a b c d"); - exec("flush -w"); - exec("insert x y z v"); - exec("flush -w"); - int oldCount = countFiles(); - // merge two files into one - exec("compact -t c -w"); - assertTrue(countFiles() < oldCount); - exec("addsplits -t c f"); - // make two more files: - exec("insert m 1 2 3"); - exec("flush -w"); - exec("insert n 1 2 3"); - exec("flush -w"); - oldCount = countFiles(); - // at this point there are 3 files in the default tablet - // compact some data: - exec("compact -b g -e z -w"); - assertTrue(countFiles() == oldCount - 2); - exec("compact -w"); - assertTrue(countFiles() == oldCount - 2); - exec("merge --all -t c"); - exec("compact -w"); - assertTrue(countFiles() == oldCount - 3); - exec("deletetable -f orig"); - exec("deletetable -f clone"); - exec("deletetable -f c"); - } - - @Test(timeout = 30000) - public void constraint() throws Exception { - // constraint - exec("constraint -l -t !METADATA", true, "MetadataConstraints=1", true); - exec("createtable c -evc"); - exec("constraint -l -t c", true, "VisibilityConstraint=2", true); - exec("constraint -t c -d 2", true, "Removed constraint 2 from table c"); - exec("constraint -l -t c", true, "VisibilityConstraint=2", false); - exec("deletetable -f c"); - } - - @Test(timeout = 30000) - public void deletemany() throws Exception { - // deletemany - exec("createtable t"); - make10(); - assertEquals(10, countkeys("t")); - exec("deletemany -f -b row8"); - assertEquals(8, countkeys("t")); - exec("scan -t t -np", true, "row8", false); - make10(); - exec("deletemany -f -b row4 -e row5"); - assertEquals(8, countkeys("t")); - make10(); - exec("deletemany -f -c cf:col4,cf:col5"); - assertEquals(8, countkeys("t")); - make10(); - exec("deletemany -f -r row3"); - assertEquals(9, countkeys("t")); - make10(); - exec("deletemany -f -r row3"); - assertEquals(9, countkeys("t")); - make10(); - exec("deletemany -f -b row3 -be -e row5 -ee"); - assertEquals(9, countkeys("t")); - exec("deletetable -f t"); - } - - @Test(timeout = 30000) - public void deleterows() throws Exception { - // deleterows - int base = countFiles(); - exec("createtable t"); - exec("addsplits row5 row7"); - make10(); - exec("flush -w -t t"); - assertTrue(base + 3 == countFiles()); - exec("deleterows -t t -b row5 -e row7", true); - assertTrue(base + 2 == countFiles()); - exec("deletetable -f t"); - } - - @Test(timeout = 30000) - public void groups() throws Exception { - exec("createtable t"); - exec("setgroups -t t alpha=a,b,c num=3,2,1"); - exec("getgroups -t t", true, "alpha=a,b,c", true); - exec("getgroups -t t", true, "num=1,2,3", true); - exec("deletetable -f t"); - } - - @Test(timeout = 30000) - public void grep() throws Exception { - exec("createtable t", true); - make10(); - exec("grep row[123]", true, "row1", false); - exec("grep row5", true, "row5", true); - exec("deletetable -f t", true); - } - - @Test - // (timeout = 30000) - public void help() throws Exception { - exec("help -np", true, "Help Commands", true); - exec("?", true, "Help Commands", true); - for (String c : ("bye exit quit " + "about help info ? " + "deleteiter deletescaniter listiter setiter setscaniter " - + "grant revoke systempermissions tablepermissions userpermissions " + "execfile history " + "authenticate cls clear notable sleep table user whoami " - + "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables " - + "addsplits compact constraint flush getgropus getsplits merge setgroups " + "addauths createuser deleteuser dropuser getauths passwd setauths users " - + "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan").split(" ")) { - exec("help " + c, true); - } - } - - // @Test(timeout = 30000) - public void history() throws Exception { - exec("history -c", true); - exec("createtable unusualstring"); - exec("deletetable -f unusualstring"); - exec("history", true, "unusualstring", true); - exec("history", true, "history", true); - } - - @Test(timeout = 30000) - public void importDirectory() throws Exception { - Configuration conf = new Configuration(); - FileSystem fs = FileSystem.get(conf); - File importDir = folder.newFolder("import"); - String even = new File(importDir, "even.rf").toString(); - String odd = new File(importDir, "odd.rf").toString(); - File errorsDir = folder.newFolder("errors"); - fs.mkdirs(new Path(errorsDir.toString())); - AccumuloConfiguration aconf = AccumuloConfiguration.getDefaultConfiguration(); - FileSKVWriter evenWriter = FileOperations.getInstance().openWriter(even, fs, conf, aconf); - evenWriter.startDefaultLocalityGroup(); - FileSKVWriter oddWriter = FileOperations.getInstance().openWriter(odd, fs, conf, aconf); - oddWriter.startDefaultLocalityGroup(); - long ts = System.currentTimeMillis(); - Text cf = new Text("cf"); - Text cq = new Text("cq"); - Value value = new Value("value".getBytes()); - for (int i = 0; i < 100; i += 2) { - Key key = new Key(new Text(String.format("%8d", i)), cf, cq, ts); - evenWriter.append(key, value); - key = new Key(new Text(String.format("%8d", i + 1)), cf, cq, ts); - oddWriter.append(key, value); - } - evenWriter.close(); - oddWriter.close(); - exec("createtable t", true); - exec("importdirectory " + importDir + " " + errorsDir + " true", true); - exec("scan -r 00000000", true, "00000000", true); - exec("scan -r 00000099", true, "00000099", true); - exec("deletetable -f t"); - } - - @Test(timeout = 30000) - public void info() throws Exception { - exec("info", true, Constants.VERSION, true); - } - - @Test(timeout = 30000) - public void interpreter() throws Exception { - exec("createtable t", true); - exec("interpreter -l", true, "HexScan", false); - exec("insert \\x02 cf cq value", true); - exec("scan -b 02", true, "value", false); - exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true); - exec("interpreter -l", true, "HexScan", true); - exec("scan -b 02", true, "value", true); - exec("deletetable -f t", true); - } - - @Test(timeout = 30000) - public void listcompactions() throws Exception { - exec("createtable t", true); - exec("config -t t -s table.iterator.minc.slow=30,org.apache.accumulo.test.functional.SlowIterator", true); - exec("config -t t -s table.iterator.minc.slow.opt.sleepTime=100", true); - exec("insert a cf cq value", true); - exec("insert b cf cq value", true); - exec("insert c cf cq value", true); - exec("insert d cf cq value", true); - exec("flush -t t", true); - exec("sleep 0.2", true); - exec("listcompactions", true, "default_tablet"); - String[] lines = output.get().split("\n"); - String last = lines[lines.length - 1]; - String[] parts = last.split("\\|"); - assertEquals(12, parts.length); - exec("deletetable -f t", true); - } - - @Test(timeout = 30000) - public void maxrow() throws Exception { - exec("createtable t", true); - exec("insert a cf cq value", true); - exec("insert b cf cq value", true); - exec("insert ccc cf cq value", true); - exec("insert zzz cf cq value", true); - exec("maxrow", true, "zzz", true); - exec("delete zzz cf cq", true); - exec("maxrow", true, "ccc", true); - exec("deletetable -f t", true); - } - - @Test(timeout = 30000) - public void merge() throws Exception { - exec("createtable t"); - exec("addsplits a m z"); - exec("getsplits", true, "z", true); - exec("merge --all", true); - exec("getsplits", true, "z", false); - exec("deletetable -f t"); - exec("getsplits -t !METADATA", true); - assertEquals(2, output.get().split("\n").length); - exec("getsplits -t !!ROOT", true); - assertEquals(1, output.get().split("\n").length); - exec("merge --all -t !METADATA"); - exec("getsplits -t !METADATA", true); - assertEquals(1, output.get().split("\n").length); - } - - @Test(timeout = 30000) - public void ping() throws Exception { - for (int i = 0; i < 10; i++) { - exec("ping", true, "OK", true); - // wait for both tservers to start up - if (output.get().split("\n").length == 3) - break; - UtilWaitThread.sleep(1000); - - } - assertEquals(3, output.get().split("\n").length); - } - - @Test(timeout = 30000) - public void renametable() throws Exception { - exec("createtable aaaa"); - exec("insert this is a value"); - exec("renametable aaaa xyzzy"); - exec("tables", true, "xyzzy", true); - exec("tables", true, "aaaa", false); - exec("scan -t xyzzy", true, "value", true); - exec("deletetable -f xyzzy", true); - } - - @Test(timeout = 30000) - public void systempermission() throws Exception { - exec("systempermissions"); - assertEquals(8, output.get().split("\n").length - 1); - exec("tablepermissions", true); - assertEquals(6, output.get().split("\n").length - 1); - } - - @Test(timeout = 30000) - public void listscans() throws Exception { - exec("createtable t", true); - exec("config -t t -s table.iterator.scan.slow=30,org.apache.accumulo.test.functional.SlowIterator", true); - exec("config -t t -s table.iterator.scan.slow.opt.sleepTime=100", true); - exec("insert a cf cq value", true); - exec("insert b cf cq value", true); - exec("insert c cf cq value", true); - exec("insert d cf cq value", true); - Thread thread = new Thread() { - @Override - public void run() { - try { - Connector connector = cluster.getConnector("root", secret); - Scanner s = connector.createScanner("t", Authorizations.EMPTY); - for (@SuppressWarnings("unused") - Entry kv : s) - ; - } catch (Exception ex) { - throw new RuntimeException(ex); - } - } - }; - thread.start(); - exec("sleep 0.1", true); - String scans = exec("listscans", true); - String lines[] = scans.split("\n"); - String last = lines[lines.length - 1]; - assertTrue(last.contains("RUNNING")); - String parts[] = last.split("\\|"); - assertEquals(13, parts.length); - thread.join(); - exec("deletetable -f t", true); - } - - @Test(timeout = 30000) - public void testPertableClasspath() throws Exception { - File fooFilterJar = File.createTempFile("FooFilter", ".jar"); - FileUtils.copyURLToFile(this.getClass().getResource("/FooFilter.jar"), fooFilterJar); - fooFilterJar.deleteOnExit(); - - File fooConstraintJar = File.createTempFile("FooConstraint", ".jar"); - FileUtils.copyURLToFile(this.getClass().getResource("/FooConstraint.jar"), fooConstraintJar); - fooConstraintJar.deleteOnExit(); - - exec( - "config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1=" + fooFilterJar.toURI().toString() + "," + fooConstraintJar.toURI().toString(), - true); - - exec("createtable ptc", true); - exec("config -t ptc -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true); - - UtilWaitThread.sleep(200); - - exec("setiter -scan -class org.apache.accumulo.test.FooFilter -p 10 -n foo", true); - - exec("insert foo f q v", true); - - UtilWaitThread.sleep(100); - - exec("scan -np", true, "foo", false); - - exec("constraint -a FooConstraint", true); - - exec("offline ptc"); - UtilWaitThread.sleep(500); - exec("online ptc"); - - exec("table ptc", true); - exec("insert foo f q v", false); - exec("insert ok foo q v", true); - - exec("deletetable ptc", true); - exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1"); - - } - - @Test(timeout = 30000) - public void trace() throws Exception { - exec("trace on", true); - exec("createtable t", true); - exec("insert a b c value", true); - exec("scan -np", true, "value", true); - exec("deletetable -f t"); - exec("sleep 1"); - String trace = exec("trace off"); - assertTrue(trace.contains("binMutations")); - assertTrue(trace.contains("update")); - assertTrue(trace.contains("DeleteTable")); - } - - private int countkeys(String table) throws IOException { - exec("scan -np -t " + table); - return output.get().split("\n").length - 1; - } - - private void make10() throws IOException { - for (int i = 0; i < 10; i++) { - exec(String.format("insert row%d cf col%d value", i, i)); - } - } - - private int countFiles() throws IOException { - exec("scan -t !METADATA -np -c file"); - return output.get().split("\n").length - 1; - } - -}