You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/01/06 20:46:30 UTC

svn commit: r1228367 - in /incubator/accumulo/trunk: src/server/src/main/java/org/apache/accumulo/server/tabletserver/ src/server/src/main/java/org/apache/accumulo/server/test/ test/system/auto/simple/

Author: vines
Date: Fri Jan  6 19:46:29 2012
New Revision: 1228367

URL: http://svn.apache.org/viewvc?rev=1228367&view=rev
Log:
Accumulo-149 - Fixing some tests
Accumulo-228 - updated DefaultMap overhead value & test

Modified:
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/EstimateInMemMapOverhead.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java
    incubator/accumulo/trunk/test/system/auto/simple/bulk.py

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java?rev=1228367&r1=1228366&r2=1228367&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java Fri Jan  6 19:46:29 2012
@@ -275,7 +275,7 @@ public class InMemoryMap {
       // and is in bytes.. the estimates were obtained by running
       // java 6_16 in 64 bit server mode
       
-      return 270;
+      return 200;
     }
     
     @Override

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/EstimateInMemMapOverhead.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/EstimateInMemMapOverhead.java?rev=1228367&r1=1228366&r2=1228367&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/EstimateInMemMapOverhead.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/EstimateInMemMapOverhead.java Fri Jan  6 19:46:29 2012
@@ -21,6 +21,7 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.server.tabletserver.InMemoryMap;
 import org.apache.hadoop.io.Text;
 
@@ -130,20 +131,23 @@ class InMemoryMapMemoryUsageTest extends
   private int keyLen;
   private int colFamLen;
   private int colQualLen;
+  private int colVisLen;
   private int dataLen;
   
   private InMemoryMap imm;
   private Text key;
   private Text colf;
   private Text colq;
+  private ColumnVisibility colv;
   private int passes;
   
-  InMemoryMapMemoryUsageTest(int passes, int keyLen, int colFamLen, int colQualLen, int dataLen) {
+  InMemoryMapMemoryUsageTest(int passes, int keyLen, int colFamLen, int colQualLen, int colVisLen, int dataLen) {
     this.keyLen = keyLen;
     this.colFamLen = colFamLen;
     this.colQualLen = colQualLen;
     this.dataLen = dataLen;
     this.passes = passes;
+    this.colVisLen = colVisLen;
     
   }
   
@@ -153,6 +157,7 @@ class InMemoryMapMemoryUsageTest extends
     
     colf = new Text(String.format("%0" + colFamLen + "d", 0));
     colq = new Text(String.format("%0" + colQualLen + "d", 0));
+    colv = new ColumnVisibility(String.format("%0" + colVisLen + "d", 0));
   }
   
   public void addEntry(int i) {
@@ -166,14 +171,14 @@ class InMemoryMapMemoryUsageTest extends
     }
     Value idata = new Value(data);
     
-    m.put(colf, colq, idata);
+    m.put(colf, colq, colv, idata);
     
     imm.mutate(Collections.singletonList(m));
     
   }
   
   public int getEstimatedBytesPerEntry() {
-    return keyLen + colFamLen + colQualLen + dataLen;
+    return keyLen + colFamLen + colQualLen + dataLen + 4 + colVisLen;
   }
   
   public void clear() {
@@ -307,31 +312,31 @@ class IntObjectMemoryUsageTest extends M
 
 public class EstimateInMemMapOverhead {
   
-  private static void runTest(int numEntries, int keyLen, int colFamLen, int colQualLen, int dataLen) {
+  private static void runTest(int numEntries, int keyLen, int colFamLen, int colQualLen, int colVisLen, int dataLen) {
     new IntObjectMemoryUsageTest(numEntries).run();
-    new InMemoryMapMemoryUsageTest(numEntries, keyLen, colFamLen, colQualLen, dataLen).run();
+    new InMemoryMapMemoryUsageTest(numEntries, keyLen, colFamLen, colQualLen, colVisLen, dataLen).run();
     new TextMemoryUsageTest(numEntries, keyLen, colFamLen, colQualLen, dataLen).run();
     new MutationMemoryUsageTest(numEntries, keyLen, colFamLen, colQualLen, dataLen).run();
   }
   
   public static void main(String[] args) {
-    runTest(10000, 10, 4, 4, 20);
-    runTest(100000, 10, 4, 4, 20);
-    runTest(500000, 10, 4, 4, 20);
-    runTest(1000000, 10, 4, 4, 20);
-    runTest(2000000, 10, 4, 4, 20);
-    
-    runTest(10000, 20, 5, 5, 500);
-    runTest(100000, 20, 5, 5, 500);
-    runTest(500000, 20, 5, 5, 500);
-    runTest(1000000, 20, 5, 5, 500);
-    runTest(2000000, 20, 5, 5, 500);
-    
-    runTest(10000, 40, 10, 10, 1000);
-    runTest(100000, 40, 10, 10, 1000);
-    runTest(500000, 40, 10, 10, 1000);
-    runTest(1000000, 40, 10, 10, 1000);
-    runTest(2000000, 40, 10, 10, 1000);
+    runTest(10000, 10, 4, 4, 4, 20);
+    runTest(100000, 10, 4, 4, 4, 20);
+    runTest(500000, 10, 4, 4, 4, 20);
+    runTest(1000000, 10, 4, 4, 4, 20);
+    runTest(2000000, 10, 4, 4, 4, 20);
+    
+    runTest(10000, 20, 5, 5, 5, 500);
+    runTest(100000, 20, 5, 5, 5, 500);
+    runTest(500000, 20, 5, 5, 5, 500);
+    runTest(1000000, 20, 5, 5, 5, 500);
+    runTest(2000000, 20, 5, 5, 5, 500);
+    
+    runTest(10000, 40, 10, 10, 10, 1000);
+    runTest(100000, 40, 10, 10, 10, 1000);
+    runTest(500000, 40, 10, 10, 10, 1000);
+    runTest(1000000, 40, 10, 10, 10, 1000);
+    runTest(2000000, 40, 10, 10, 10, 1000);
   }
   
 }

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java?rev=1228367&r1=1228366&r2=1228367&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java Fri Jan  6 19:46:29 2012
@@ -123,7 +123,6 @@ public class TestIngest {
     
     public boolean unique;
     
-    boolean outputToMapFile = false;
     boolean outputToRFile = false;
     String outputFile;
     
@@ -147,9 +146,8 @@ public class TestIngest {
     opts.addOption(new Option("tsbw", "tsbw", false, "tsbw"));
     opts.addOption(new Option("username", "username", true, "username"));
     opts.addOption(new Option("password", "password", true, "password"));
-    opts.addOption(new Option("mapFile", "mapFile", true, "map file"));
     opts.addOption(new Option("trace", "trace", false, "turn on distributed tracing"));
-    opts.addOption(new Option("rfile", "rfile", true, "relative-key file"));
+    opts.addOption(new Option("rFile", "rFile", true, "relative-key file"));
     return opts;
   }
   
@@ -184,18 +182,9 @@ public class TestIngest {
       ia.timestamp = Long.parseLong(cl.getOptionValue("timestamp"));
       ia.hasTimestamp = true;
     }
-    if (cl.hasOption("mapFile")) {
-      ia.outputToMapFile = true;
-      ia.outputFile = cl.getOptionValue("mapFile");
-    }
-    if (cl.hasOption("rfile")) {
+    if (cl.hasOption("rFile")) {
       ia.outputToRFile = true;
-      ia.outputFile = cl.getOptionValue("rfile");
-    }
-    if (ia.outputToMapFile && ia.outputToRFile) {
-      HelpFormatter hf = new HelpFormatter();
-      hf.printHelp("Cannot output to both an rfile and a map file", getOptions());
-      throw new RuntimeException();
+      ia.outputFile = cl.getOptionValue("rFile");
     }
     ia.delete = cl.hasOption("delete");
     ia.useGet = cl.hasOption("useGet");
@@ -291,13 +280,7 @@ public class TestIngest {
       FileSKVWriter writer = null;
       
       rootCredentials = new AuthInfo(username, ByteBuffer.wrap(passwd.getBytes()), instance.getInstanceID());
-      if (ingestArgs.outputToMapFile) {
-        Configuration conf = CachedConfiguration.getInstance();
-        FileSystem fs = FileSystem.get(conf);
-        writer = FileOperations.getInstance().openWriter(ingestArgs.outputFile + "." + RFile.EXTENSION, fs, conf,
-            AccumuloConfiguration.getDefaultConfiguration());
-        writer.startDefaultLocalityGroup();
-      } else if (ingestArgs.outputToRFile) {
+      if (ingestArgs.outputToRFile) {
         Configuration conf = CachedConfiguration.getInstance();
         FileSystem fs = FileSystem.get(conf);
         writer = FileOperations.getInstance().openWriter(ingestArgs.outputFile + "." + RFile.EXTENSION, fs, conf,

Modified: incubator/accumulo/trunk/test/system/auto/simple/bulk.py
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/test/system/auto/simple/bulk.py?rev=1228367&r1=1228366&r2=1228367&view=diff
==============================================================================
--- incubator/accumulo/trunk/test/system/auto/simple/bulk.py (original)
+++ incubator/accumulo/trunk/test/system/auto/simple/bulk.py Fri Jan  6 19:46:29 2012
@@ -37,33 +37,33 @@ class SimpleBulkTest(TestUtilsMixin, uni
                              args,
                              **kwargs)
 
-    def bulkLoad(self, host):
+    def bulkLoad(self, host, dir):
         handle = self.runClassOn(
             self.masterHost(),
             'org.apache.accumulo.server.test.BulkImportDirectory',
             [ROOT, ROOT_PASSWORD,
-             'test_ingest', '/testmf', '/testmfFail'])
+             'test_ingest', dir, '/testBulkFail'])
         self.wait(handle)
         self.assert_(handle.returncode == 0)
         
 
-    def createMapFiles(self):
-        args = '-mapFile /testmf/mf%02d -timestamp 1 -size 50 -random 56 %1d %ld 1'
-        log.info('creating map files')
+    def createRFiles(self):
+        args = '-rFile /testrf/rf%02d -timestamp 1 -size 50 -random 56 %1d %ld 1'
+        log.info('creating rfiles')
         handles = []
         for i in range(COUNT):
             handles.append(self.testIngest(
                 self.hosts[i%len(self.hosts)],
                 (args % (i, N, (N * i))).split()))
         
-        #create a map file with one entry, there was a bug with this
+        #create a rfile with one entry, there was a bug with this
         handles.append(self.testIngest(self.hosts[0], (args % (COUNT, 1, COUNT * N)).split()))
         log.info('waiting to finish')
         for h in handles:
             h.communicate()
             self.assert_(h.returncode == 0)
         log.info('done')
-
+        
     def execute(self, host, cmd, **opts):
         handle = self.runOn(host, cmd, **opts)
         out, err = handle.communicate()
@@ -73,14 +73,14 @@ class SimpleBulkTest(TestUtilsMixin, uni
 
         # initialize the database
         self.createTable('test_ingest')
-        self.execute(self.masterHost(), 'hadoop dfs -rmr /testmf'.split())
-        self.execute(self.masterHost(), 'hadoop dfs -rmr /testmfFail'.split())
-        self.execute(self.masterHost(), 'hadoop dfs -mkdir /testmfFail'.split())
+        self.execute(self.masterHost(), 'hadoop dfs -rmr /testrf'.split())
+        self.execute(self.masterHost(), 'hadoop dfs -rmr /testBulkFail'.split())
+        self.execute(self.masterHost(), 'hadoop dfs -mkdir /testBulkFail'.split())
 
         # insert some data
-        self.createMapFiles()
-        self.bulkLoad(self.masterHost())
-
+        self.createRFiles()
+        self.bulkLoad(self.masterHost(), '/testrf')
+        
         log.info("Verifying Ingestion")
         handles = []
         for i in range(COUNT):