You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/07/03 23:21:48 UTC

svn commit: r1499562 - in /accumulo/trunk/test: src/main/java/org/apache/accumulo/test/ src/main/java/org/apache/accumulo/test/functional/ src/test/java/org/apache/accumulo/test/functional/ system/auto/stress/

Author: ecn
Date: Wed Jul  3 21:21:48 2013
New Revision: 1499562

URL: http://svn.apache.org/r1499562
Log:
ACCUMULO-1537 started the stress tests, fixed some tests, made others more reliable

Added:
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java
      - copied, changed from r1499109, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateManyScannersTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java   (with props)
Removed:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateManyScannersTest.java
    accumulo/trunk/test/system/auto/stress/batchWrite.py
    accumulo/trunk/test/system/auto/stress/bigRootTablet.py
    accumulo/trunk/test/system/auto/stress/binary.py
    accumulo/trunk/test/system/auto/stress/manyScanners.py
    accumulo/trunk/test/system/auto/stress/metadataMaxFiles.py
Modified:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java Wed Jul  3 21:21:48 2013
@@ -54,7 +54,7 @@ public class TestMultiTableIngest {
       scanner.setBatchSize(scanOpts.scanBatchSize);
       int count = i;
       for (Entry<Key,Value> elt : scanner) {
-        String expected = String.format("%05d", count);
+        String expected = String.format("%06d", count);
         if (!elt.getKey().getRow().toString().equals(expected))
           throw new RuntimeException("entry " + elt + " does not match expected " + expected + " in table " + table);
         count += tableNames.size();
@@ -96,7 +96,7 @@ public class TestMultiTableIngest {
       
       // populate
       for (int i = 0; i < opts.count; i++) {
-        Mutation m = new Mutation(new Text(String.format("%05d", i)));
+        Mutation m = new Mutation(new Text(String.format("%06d", i)));
         m.put(new Text("col" + Integer.toString((i % 3) + 1)), new Text("qual"), new Value("junk".getBytes()));
         b.getBatchWriter(tableNames.get(i % tableNames.size())).addMutation(m);
       }

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java?rev=1499562&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java Wed Jul  3 21:21:48 2013
@@ -0,0 +1,63 @@
+/*
+ * 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.*;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.Connector;
+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.security.Authorizations;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.accumulo.core.util.RootTable;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.junit.Test;
+
+public class BigRootTabletIT extends MacTest {
+  // ACCUMULO-542: A large root tablet will fail to load if it does't fit in the tserver scan buffers
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.TABLE_SCAN_MAXMEM.getKey(), "1024");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "60m");
+    cfg.setSiteConfig(siteConfig );
+  }
+
+  @Test(timeout=60*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().addSplits(MetadataTable.NAME, FunctionalTestUtils.splits("0 1 2 3 4 5 6 7 8 9 a".split(" ")));
+    for (int i = 0; i < 10; i++) {
+      c.tableOperations().create("" + i);
+      c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+      c.tableOperations().flush(RootTable.NAME, null, null, true);
+    }
+    cluster.stop();
+    cluster.start();
+    int count = 0;
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry : c.createScanner(RootTable.NAME, Authorizations.EMPTY))
+      count++;
+    assertTrue(count > 0);
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java Wed Jul  3 21:21:48 2013
@@ -45,7 +45,7 @@ public class BinaryIT extends MacTest {
     runTest(c);
   }
   
-  void runTest(Connector c) throws Exception {
+  public static void runTest(Connector c) throws Exception {
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     ScannerOpts scanOpts = new ScannerOpts();
     TestBinaryRows.Opts opts = new TestBinaryRows.Opts();

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java?rev=1499562&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java Wed Jul  3 21:21:48 2013
@@ -0,0 +1,55 @@
+/*
+ * 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.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class BinaryStressIT extends MacTest {
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    Map<String,String> siteConfig = new HashMap<String,String>();
+    siteConfig.put(Property.TSERV_MAXMEM.getKey(), "50K");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
+    cfg.setSiteConfig(siteConfig );
+  }
+
+  @Test(timeout=60*1000)
+  public void binaryStressTest() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("bt");
+    c.tableOperations().setProperty("bt", Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
+    BinaryIT.runTest(c);
+    String id = c.tableOperations().tableIdMap().get("bt");
+    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    FileStatus[] dir = fs.listStatus(new Path(cluster.getConfig().getDir() + "/accumulo/tables/" + id));
+    assertTrue(dir.length  > 7);
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java Wed Jul  3 21:21:48 2013
@@ -49,15 +49,15 @@ public class BloomFilterIT extends MacTe
     cfg.setDefaultMemory(500, MemoryUnit.MEGABYTE);
   }
   
-  @Test(timeout=120*1000)
+  @Test(timeout=500*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     for (String table : "bt1 bt2 bt3 bt4".split(" ")) {
       c.tableOperations().create(table);
     }
-    write(c, "bt1", 1, 0, 1000000000, 500);
-    write(c, "bt2", 2, 0, 1000000000, 500);
-    write(c, "bt3", 3, 0, 1000000000, 500);
+    write(c, "bt1", 1, 0, 1000000000, 100);
+    write(c, "bt2", 2, 0, 1000000000, 100);
+    write(c, "bt3", 3, 0, 1000000000, 100);
     
     // test inserting an empty key
     BatchWriter bw = c.createBatchWriter("bt4", new BatchWriterConfig());
@@ -80,9 +80,9 @@ public class BloomFilterIT extends MacTe
     FunctionalTestUtils.checkRFiles(c, "bt4", 1, 1, 1, 1);
     
     // these queries should only run quickly if bloom filters are working, so lets get a base
-    long t1 = query(c, "bt1", 1, 0, 1000000000, 100000, 500);
-    long t2 = query(c, "bt2", 2, 0, 1000000000, 100000, 500);
-    long t3 = query(c, "bt3", 3, 0, 1000000000, 100000, 500);
+    long t1 = query(c, "bt1", 1, 0, 1000000000, 100000, 100);
+    long t2 = query(c, "bt2", 2, 0, 1000000000, 100000, 100);
+    long t3 = query(c, "bt3", 3, 0, 1000000000, 100000, 100);
     
     c.tableOperations().setProperty("bt1", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
     c.tableOperations().setProperty("bt1", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), RowFunctor.class.getName());
@@ -102,9 +102,9 @@ public class BloomFilterIT extends MacTe
     
     // these queries should only run quickly if bloom
     // filters are working
-    long tb1 = query(c, "bt1", 1, 0, 1000000000, 100000, 500);
-    long tb2 = query(c, "bt2", 2, 0, 1000000000, 100000, 500);
-    long tb3 = query(c, "bt3", 3, 0, 1000000000, 100000, 500);
+    long tb1 = query(c, "bt1", 1, 0, 1000000000, 100000, 100);
+    long tb2 = query(c, "bt2", 2, 0, 1000000000, 100000, 100);
+    long tb3 = query(c, "bt3", 3, 0, 1000000000, 100000, 100);
     
     timeCheck(t1, tb1);
     timeCheck(t2, tb2);

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java?rev=1499562&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java Wed Jul  3 21:21:48 2013
@@ -0,0 +1,65 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.server.master.balancer.ChaoticLoadBalancer;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class ChaoticBlancerIT extends MacTest {
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
+    siteConfig.put(Property.TABLE_LOAD_BALANCER.getKey(), ChaoticLoadBalancer.class.getName());
+    cfg.setSiteConfig(siteConfig );
+  }
+
+  @Test(timeout=120*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("test_ingest");
+    c.tableOperations().setProperty("test_ingest", Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
+    SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 0; i < 200; i++) {
+      splits.add(new Text(String.format("%03d", i)));
+    }
+    c.tableOperations().create("unused");
+    c.tableOperations().addSplits("unused", splits);
+    TestIngest.Opts opts = new TestIngest.Opts();
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.rows = opts.rows = 200000;
+    TestIngest.ingest(c, opts, new BatchWriterOpts());
+    c.tableOperations().flush("test_ingest", null, null, true);
+    VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBlancerIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java (from r1499109, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateManyScannersTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateManyScannersTest.java&r1=1499109&r2=1499562&rev=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateManyScannersTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java Wed Jul  3 21:21:48 2013
@@ -16,35 +16,18 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.Authorizations;
+import org.junit.Test;
 
-public class CreateManyScannersTest extends FunctionalTest {
-  
-  @Override
-  public void cleanup() throws Exception {
-    
-  }
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup("mscant"));
-  }
+public class CreateManyScannersIT extends MacTest {
   
-  @Override
+  @Test(timeout=10*1000)
   public void run() throws Exception {
-    Connector connector = getConnector();
+    Connector c = getConnector();
+    c.tableOperations().create("mscant");
     for (int i = 0; i < 100000; i++) {
-      connector.createScanner("mscant", Authorizations.EMPTY);
+      c.createScanner("mscant", Authorizations.EMPTY);
     }
   }
   

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java Wed Jul  3 21:21:48 2013
@@ -23,8 +23,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.SortedSet;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -160,5 +162,11 @@ public class FunctionalTestUtils {
   static Mutation nm(String row, String cf, String cq, String value) {
     return nm(row, cf, cq, new Value(value.getBytes()));
   }
+  public static SortedSet<Text> splits(String [] splits) {
+    SortedSet<Text> result = new TreeSet<Text>();
+    for (String split : splits)
+      result.add(new Text(split));
+    return result;
+  }
   
 }

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java?rev=1499562&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java Wed Jul  3 21:21:48 2013
@@ -0,0 +1,108 @@
+/*
+ * 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.*;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.accumulo.core.util.RootTable;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class MetadataMaxFiles extends MacTest {
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1");
+    siteConfig.put(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), "10");
+    cfg.setSiteConfig(siteConfig );
+  }
+
+  @Test(timeout=240*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 0; i < 1000; i++) {
+      splits.add(new Text(String.format("%03d", i)));
+    }
+    c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10000");
+    for (int i = 0; i < 5; i++) {
+      String tableName = "table" + i;
+      log.info("Creating " + tableName);
+      c.tableOperations().create(tableName);
+      log.info("adding splits");
+      c.tableOperations().addSplits(tableName, splits);
+      log.info("flushing");
+      c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+      c.tableOperations().flush(RootTable.NAME, null, null, true);
+    }
+    UtilWaitThread.sleep(20*1000);
+    log.info("shutting down");
+    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
+    cluster.stop();
+    log.info("starting up");
+    cluster.start();
+
+    UtilWaitThread.sleep(30*1000);
+
+    while (true) {
+      MasterMonitorInfo stats = null;
+      TCredentials creds = CredentialHelper.create("root", new PasswordToken(MacTest.PASSWORD), c.getInstance().getInstanceName());
+      Client client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(c.getInstance());
+        stats = client.getMasterStats(Tracer.traceInfo(), creds);
+      } finally {
+        if (client != null)
+          MasterClient.close(client);
+      }
+      int tablets = 0;
+      for (TabletServerStatus tserver : stats.tServerInfo) {
+        for (Entry<String,TableInfo> entry : tserver.tableMap.entrySet()) {
+          if (entry.getKey().startsWith("!"))
+            continue;
+          tablets += entry.getValue().onlineTablets;
+        }
+      }
+      if (tablets == 5005)
+        break;
+      UtilWaitThread.sleep(1000);
+    }
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MetadataMaxFiles.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java Wed Jul  3 21:21:48 2013
@@ -57,7 +57,7 @@ import org.junit.Test;
 
 public class ReadWriteIT extends MacTest {
   
-  static final int ROWS = 20000;
+  static final int ROWS = 200000;
   static final int COLS = 1;
   static final String COLF = "colf";
   

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java Wed Jul  3 21:21:48 2013
@@ -48,7 +48,7 @@ public class SplitIT extends MacTest {
   public void configure(MiniAccumuloConfig cfg) {
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "5K");
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1s");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "100ms");
     cfg.setSiteConfig(siteConfig);
   }
   
@@ -92,8 +92,8 @@ public class SplitIT extends MacTest {
     c.tableOperations().setProperty("test_ingest", Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
     c.tableOperations().setProperty("test_ingest", Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), "none");
     ReadWriteIT.interleaveTest(c);
-    UtilWaitThread.sleep(5 * 1000);
-    assertTrue(c.tableOperations().listSplits("test_ingest").size() > 10);
+    UtilWaitThread.sleep(5*1000);
+    assertTrue(c.tableOperations().listSplits("test_ingest").size() > 20);
   }
   
   @Test(timeout = 120 * 1000)
@@ -102,6 +102,8 @@ public class SplitIT extends MacTest {
     c.tableOperations().create("test_ingest");
     c.tableOperations().setProperty("test_ingest", Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
     DeleteIT.deleteTest(c);
+    c.tableOperations().flush("test_ingest", null, null, true);
+    UtilWaitThread.sleep(5*1000);
     assertTrue(c.tableOperations().listSplits("test_ingest").size() > 30);
   }
   

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java?rev=1499562&r1=1499561&r2=1499562&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java Wed Jul  3 21:21:48 2013
@@ -42,7 +42,7 @@ import org.junit.Test;
  */
 public class TimeoutIT extends MacTest {
   
-  @Test
+  @Test(timeout=30*1000)
   public void run() throws Exception {
     Connector conn = getConnector();
     testBatchWriterTimeout(conn);

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java?rev=1499562&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java Wed Jul  3 21:21:48 2013
@@ -0,0 +1,66 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.junit.Test;
+
+public class WriteLotsIT extends MacTest {
+  
+  @Test(timeout=20*1000)
+  public void writeLots() throws Exception {
+    final Connector c = getConnector();
+    c.tableOperations().create("test_ingest");
+    final AtomicReference<Exception> ref = new AtomicReference<Exception>();
+    List<Thread> threads = new ArrayList<Thread>();
+    for (int i = 0; i < 10; i++) {
+      final int index = i;
+      Thread t = new Thread() {
+        public void run() {
+          try {
+            TestIngest.Opts opts = new TestIngest.Opts();
+            opts.startRow = index * 10000;
+            opts.rows = 10000;
+            TestIngest.ingest(c, opts, new BatchWriterOpts());
+          } catch (Exception ex) {
+            ref.set(ex);
+          }
+        }
+      };
+      t.start();
+      threads.add(t);
+    }
+    for (Thread thread : threads) {
+      thread.join();
+    }
+    if (ref.get() != null) {
+      throw ref.get();
+    }
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.rows = 10000 * 10;
+    VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
------------------------------------------------------------------------------
    svn:eol-style = native