accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From els...@apache.org
Subject [28/50] [abbrv] accumulo git commit: ACCUMULO-3167 Get BulkSplitOptimizationIT working against standalone cluster
Date Wed, 26 Nov 2014 04:23:44 GMT
ACCUMULO-3167 Get BulkSplitOptimizationIT working against standalone cluster


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

Branch: refs/heads/1.6
Commit: b130b1d201be8053415c8a8039acc6e8ea62b1c7
Parents: 91dcc95
Author: Josh Elser <elserj@apache.org>
Authored: Mon Nov 24 13:23:27 2014 -0500
Committer: Josh Elser <elserj@apache.org>
Committed: Mon Nov 24 18:08:52 2014 -0500

----------------------------------------------------------------------
 .../functional/BulkSplitOptimizationIT.java     | 88 +++++++++++++-------
 1 file changed, 59 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b130b1d2/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
b/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 492e36f..27b5147 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -16,35 +16,37 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.io.File;
+import static com.google.common.base.Charsets.UTF_8;
+
 import java.util.Collections;
 
 import org.apache.accumulo.core.cli.ClientOpts.Password;
+import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.AccumuloClusterIT;
+import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.junit.Rule;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 /**
  * This test verifies that when a lot of files are bulk imported into a table with one tablet
and then splits that not all map files go to the children tablets.
  */
 
-public class BulkSplitOptimizationIT extends ConfigurableMacIT {
-
-  private static final String TABLE_NAME = "test_ingest";
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir")
+ "/target"));
+public class BulkSplitOptimizationIT extends AccumuloClusterIT {
 
   @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
{
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
 
@@ -53,36 +55,63 @@ public class BulkSplitOptimizationIT extends ConfigurableMacIT {
     return 2 * 60;
   }
 
+  private String majcDelay;
+
+  @Before
+  public void alterConfig() throws Exception {
+    Connector conn = getConnector();
+    majcDelay = conn.instanceOperations().getSystemConfiguration().get(Property.TSERV_MAJC_DELAY.getKey());
+    if (!"1s".equals(majcDelay)) {
+      conn.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1s");
+      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    }
+  }
+
+  @After
+  public void resetConfig() throws Exception {
+    if (null != majcDelay) {
+      Connector conn = getConnector();
+      conn.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
+      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    }
+  }
+
   static final int ROWS = 100000;
   static final int SPLITS = 99;
 
   @Test
   public void testBulkSplitOptimization() throws Exception {
     final Connector c = getConnector();
-    c.tableOperations().create(TABLE_NAME);
-    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_MAJC_RATIO.getKey(), "1000");
-    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_FILE_MAX.getKey(), "1000");
-    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
"1G");
-
-    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    FunctionalTestUtils.createRFiles(c, fs, folder.getRoot() + "/testmf", ROWS, SPLITS, 8);
-    File input = new File(folder.getRoot() + "/testmf");
-    System.out.println("Number of generated files: " + input.listFiles().length);
-    FunctionalTestUtils.bulkImport(c, fs, TABLE_NAME, folder.getRoot() + "/testmf");
-    FunctionalTestUtils.checkSplits(c, TABLE_NAME, 0, 0);
-    FunctionalTestUtils.checkRFiles(c, TABLE_NAME, 1, 1, 100, 100);
+    final String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "1000");
+    c.tableOperations().setProperty(tableName, Property.TABLE_FILE_MAX.getKey(), "1000");
+    c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "1G");
+
+    FileSystem fs = getFileSystem();
+    Path testDir = new Path(getUsableDir(), "testmf");
+    FunctionalTestUtils.createRFiles(c, fs, testDir.toString(), ROWS, SPLITS, 8);
+    FileStatus[] stats = fs.listStatus(testDir);
+    System.out.println("Number of generated files: " + stats.length);
+    FunctionalTestUtils.bulkImport(c, fs, tableName, testDir.toString());
+    FunctionalTestUtils.checkSplits(c, tableName, 0, 0);
+    FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 100, 100);
 
     // initiate splits
-    getConnector().tableOperations().setProperty(TABLE_NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
"100K");
+    getConnector().tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(),
"100K");
 
     UtilWaitThread.sleep(2000);
 
     // wait until over split threshold -- should be 78 splits
-    while (getConnector().tableOperations().listSplits(TABLE_NAME).size() < 75) {
+    while (getConnector().tableOperations().listSplits(tableName).size() < 75) {
       UtilWaitThread.sleep(500);
     }
 
-    FunctionalTestUtils.checkSplits(c, TABLE_NAME, 50, 100);
+    Assume.assumeTrue(getToken() instanceof PasswordToken);
+    PasswordToken token = (PasswordToken) getToken();
+    FunctionalTestUtils.checkSplits(c, tableName, 50, 100);
     VerifyIngest.Opts opts = new VerifyIngest.Opts();
     opts.timestamp = 1;
     opts.dataSize = 50;
@@ -90,11 +119,12 @@ public class BulkSplitOptimizationIT extends ConfigurableMacIT {
     opts.rows = 100000;
     opts.startRow = 0;
     opts.cols = 1;
-    opts.password = new Password(ROOT_PASSWORD);
-    VerifyIngest.verifyIngest(c, opts, SOPTS);
+    opts.password = new Password(new String(token.getPassword(), UTF_8));
+    opts.tableName = tableName;
+    VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
 
     // ensure each tablet does not have all map files, should be ~2.5 files per tablet
-    FunctionalTestUtils.checkRFiles(c, TABLE_NAME, 50, 100, 1, 4);
+    FunctionalTestUtils.checkRFiles(c, tableName, 50, 100, 1, 4);
   }
 
 }


Mime
View raw message