accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From els...@apache.org
Subject [3/3] accumulo git commit: Merge branch '1.7'
Date Thu, 17 Sep 2015 23:53:59 GMT
Merge branch '1.7'


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

Branch: refs/heads/master
Commit: a0c96e0ef5ed136addfffa74664f9c1de2f154f6
Parents: 3b4e57a e0621df
Author: Josh Elser <elserj@apache.org>
Authored: Thu Sep 17 19:52:56 2015 -0400
Committer: Josh Elser <elserj@apache.org>
Committed: Thu Sep 17 19:52:56 2015 -0400

----------------------------------------------------------------------
 .../test/BulkImportSequentialRowsIT.java        | 22 ++++++++++++--------
 .../accumulo/test/GenerateSequentialRFile.java  |  4 +++-
 2 files changed, 16 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0c96e0e/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
index 4430fc7,0000000..0297c14
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
@@@ -1,104 -1,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;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.util.TreeSet;
 +
 +import org.apache.accumulo.core.client.admin.TableOperations;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.harness.AccumuloClusterHarness;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.RawLocalFileSystem;
 +import org.apache.hadoop.io.Text;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterables;
 +
 +// ACCUMULO-3967
 +public class BulkImportSequentialRowsIT extends AccumuloClusterHarness {
 +  private static final Logger log = LoggerFactory.getLogger(BulkImportSequentialRowsIT.class);
 +
 +  private static final long NR = 24;
 +  private static final long NV = 42000;
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  @Override
 +  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
{
 +    // Need more than one tserver
 +    cfg.setNumTservers(2);
 +
 +    // use raw local file system so walogs sync and flush will work
 +    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
 +  }
 +
 +  @Test
 +  public void testBulkImportFailure() throws Exception {
 +    String tableName = getUniqueNames(1)[0];
 +    TableOperations to = getConnector().tableOperations();
 +    to.create(tableName);
 +    FileSystem fs = getFileSystem();
-     Path rootPath = getUsableDir();
++    Path rootPath = new Path(fs.makeQualified(getUsableDir()), getClass().getSimpleName());
++    log.info("Writing to {}", rootPath);
++    if (fs.exists(rootPath)) {
++      assertTrue(fs.delete(rootPath, true));
++    }
++    assertTrue(fs.mkdirs(rootPath));
++
 +    Path bulk = new Path(rootPath, "bulk");
 +    log.info("bulk: {}", bulk);
-     if (fs.exists(bulk)) {
-       fs.delete(bulk, true);
-     }
 +    assertTrue(fs.mkdirs(bulk));
 +    Path err = new Path(rootPath, "err");
 +    log.info("err: {}", err);
-     if (fs.exists(err)) {
-       fs.delete(err, true);
-     }
++
++    assertTrue(fs.mkdirs(bulk));
 +    assertTrue(fs.mkdirs(err));
 +
 +    Path rfile = new Path(bulk, "file.rf");
 +
-     GenerateSequentialRFile.main(new String[] {"-f", rfile.toString(), "-nr", Long.toString(NR),
"-nv", Long.toString(NV)});
++    log.info("Generating RFile {}", rfile.toUri().toString());
++
++    GenerateSequentialRFile.main(new String[] {"-f", rfile.toUri().toString(), "-nr", Long.toString(NR),
"-nv", Long.toString(NV)});
 +
-     assertTrue(fs.exists(rfile));
++    assertTrue("Expected that " + rfile + " exists, but it does not", fs.exists(rfile));
 +
 +    // Add some splits
 +    to.addSplits(tableName, getSplits());
 +
 +    // Then import a single rfile to all the tablets, hoping that we get a failure to import
because of the balancer moving tablets around
 +    // and then we get to verify that the bug is actually fixed.
 +    to.importDirectory(tableName, bulk.toString(), err.toString(), false);
 +
 +    // The bug is that some tablets don't get imported into.
 +    assertEquals(NR * NV, Iterables.size(getConnector().createScanner(tableName, Authorizations.EMPTY)));
 +  }
 +
 +  private TreeSet<Text> getSplits() {
 +    TreeSet<Text> splits = new TreeSet<>();
 +    for (int i = 0; i < NR; i++) {
 +      splits.add(new Text(String.format("%03d", i)));
 +    }
 +    return splits;
 +  }
 +}


Mime
View raw message