diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java index f260e780..d583e329 100644 --- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java +++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java @@ -177,7 +177,7 @@ private static long flush(BatchWriter bw, long count, final int flushInterval, l return lastFlushTime; } - static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility cv, + public static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility cv, byte[] ingestInstanceId, long count, byte[] prevRow, boolean checksum) { // Adler32 is supposed to be faster, but according to wikipedia is not good for small data.... so used CRC32 instead @@ -202,7 +202,7 @@ static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility return m; } - static long genLong(long min, long max, Random r) { + public static long genLong(long min, long max, Random r) { return ((r.nextLong() & 0x7fffffffffffffffL) % (max - min)) + min; } diff --git a/core/src/main/java/org/apache/accumulo/testing/core/ingest/BulkImportDirectory.java b/core/src/main/java/org/apache/accumulo/testing/core/ingest/BulkImportDirectory.java new file mode 100644 index 00000000..074bd8be --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/ingest/BulkImportDirectory.java @@ -0,0 +1,68 @@ +/* + * 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.testing.core.ingest; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.security.tokens.PasswordToken; +import org.apache.accumulo.core.util.CachedConfiguration; +import org.apache.accumulo.server.cli.ClientOnRequiredTable; +import org.apache.accumulo.server.client.HdfsZooInstance; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import com.beust.jcommander.Parameter; + +public class BulkImportDirectory { + static class Opts extends ClientOnRequiredTable { + @Parameter(names = {"-s", "--source"}, description = "directory to import from") + String source = null; + @Parameter(names = {"-f", "--failures"}, description = "directory to copy failures into: will be deleted before the bulk import") + String failures = null; + @Parameter(description = " ") + List args = new ArrayList<>(); + } + + public static void main(String[] args) throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException { + final FileSystem fs = FileSystem.get(CachedConfiguration.getInstance()); + Opts opts = new Opts(); + if (args.length == 5) { + System.err.println("Deprecated syntax for BulkImportDirectory, please use the new style (see --help)"); + final String user = args[0]; + final byte[] pass = args[1].getBytes(UTF_8); + final String tableName = args[2]; + final String dir = args[3]; + final String failureDir = args[4]; + final Path failureDirPath = new Path(failureDir); + fs.delete(failureDirPath, true); + fs.mkdirs(failureDirPath); + HdfsZooInstance.getInstance().getConnector(user, new PasswordToken(pass)).tableOperations().importDirectory(tableName, dir, failureDir, false); + } else { + opts.parseArgs(BulkImportDirectory.class.getName(), args); + fs.delete(new Path(opts.failures), true); + fs.mkdirs(new Path(opts.failures)); + opts.getConnector().tableOperations().importDirectory(opts.getTableName(), opts.source, opts.failures, false); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/ingest/TestIngest.java b/core/src/main/java/org/apache/accumulo/testing/core/ingest/TestIngest.java new file mode 100644 index 00000000..bb40f6f3 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/ingest/TestIngest.java @@ -0,0 +1,344 @@ +/* + * 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.testing.core.ingest; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import java.io.IOException; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.accumulo.core.cli.BatchWriterOpts; +import org.apache.accumulo.core.cli.ClientOnDefaultTable; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.MutationsRejectedException; +import org.apache.accumulo.core.client.TableExistsException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.impl.TabletServerBatchWriter; +import org.apache.accumulo.core.client.security.SecurityErrorCode; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.data.ConstraintViolationSummary; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.TabletId; +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.file.rfile.RFile; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.security.ColumnVisibility; +import org.apache.accumulo.core.trace.DistributedTrace; +import org.apache.accumulo.core.util.CachedConfiguration; +import org.apache.accumulo.core.util.FastFormat; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.Text; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +import com.beust.jcommander.Parameter; + +public class TestIngest { + public static final Authorizations AUTHS = new Authorizations("L1", "L2", "G1", "GROUP2"); + + public static class Opts extends ClientOnDefaultTable { + + @Parameter(names = "--createTable") + public boolean createTable = false; + + @Parameter(names = "--splits", description = "the number of splits to use when creating the table") + public int numsplits = 1; + + @Parameter(names = "--start", description = "the starting row number") + public int startRow = 0; + + @Parameter(names = "--rows", description = "the number of rows to ingest") + public int rows = 100000; + + @Parameter(names = "--cols", description = "the number of columns to ingest per row") + public int cols = 1; + + @Parameter(names = "--random", description = "insert random rows and use the given number to seed the psuedo-random number generator") + public Integer random = null; + + @Parameter(names = "--size", description = "the size of the value to ingest") + public int dataSize = 1000; + + @Parameter(names = "--delete", description = "delete values instead of inserting them") + public boolean delete = false; + + @Parameter(names = {"-ts", "--timestamp"}, description = "timestamp to use for all values") + public long timestamp = -1; + + @Parameter(names = "--rfile", description = "generate data into a file that can be imported") + public String outputFile = null; + + @Parameter(names = "--stride", description = "the difference between successive row ids") + public int stride; + + @Parameter(names = {"-cf", "--columnFamily"}, description = "place columns in this column family") + public String columnFamily = "colf"; + + @Parameter(names = {"-cv", "--columnVisibility"}, description = "place columns in this column family", converter = VisibilityConverter.class) + public ColumnVisibility columnVisibility = new ColumnVisibility(); + + public Configuration conf = null; + public FileSystem fs = null; + + public Opts() { + super("test_ingest"); + } + } + + public static void createTable(Connector conn, Opts args) throws AccumuloException, AccumuloSecurityException, TableExistsException { + if (args.createTable) { + TreeSet splits = getSplitPoints(args.startRow, args.startRow + args.rows, args.numsplits); + + if (!conn.tableOperations().exists(args.getTableName())) + conn.tableOperations().create(args.getTableName()); + try { + conn.tableOperations().addSplits(args.getTableName(), splits); + } catch (TableNotFoundException ex) { + // unlikely + throw new RuntimeException(ex); + } + } + } + + public static TreeSet getSplitPoints(long start, long end, long numsplits) { + long splitSize = (end - start) / numsplits; + + long pos = start + splitSize; + + TreeSet splits = new TreeSet<>(); + + while (pos < end) { + splits.add(new Text(String.format("row_%010d", pos))); + pos += splitSize; + } + return splits; + } + + public static byte[][] generateValues(int dataSize) { + + byte[][] bytevals = new byte[10][]; + + byte[] letters = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0'}; + + for (int i = 0; i < 10; i++) { + bytevals[i] = new byte[dataSize]; + for (int j = 0; j < dataSize; j++) + bytevals[i][j] = letters[i]; + } + return bytevals; + } + + private static byte ROW_PREFIX[] = "row_".getBytes(UTF_8); + private static byte COL_PREFIX[] = "col_".getBytes(UTF_8); + + public static Text generateRow(int rowid, int startRow) { + return new Text(FastFormat.toZeroPaddedString(rowid + startRow, 10, 10, ROW_PREFIX)); + } + + public static byte[] genRandomValue(Random random, byte dest[], int seed, int row, int col) { + random.setSeed((row ^ seed) ^ col); + random.nextBytes(dest); + toPrintableChars(dest); + + return dest; + } + + public static void toPrintableChars(byte[] dest) { + // transform to printable chars + for (int i = 0; i < dest.length; i++) { + dest[i] = (byte) (((0xff & dest[i]) % 92) + ' '); + } + } + + public static void main(String[] args) throws Exception { + + Opts opts = new Opts(); + BatchWriterOpts bwOpts = new BatchWriterOpts(); + opts.parseArgs(TestIngest.class.getName(), args, bwOpts); + + String name = TestIngest.class.getSimpleName(); + DistributedTrace.enable(name); + + try { + opts.startTracing(name); + + if (opts.debug) + Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE); + + // test batch update + + ingest(opts.getConnector(), opts, bwOpts); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + opts.stopTracing(); + DistributedTrace.disable(); + } + } + + public static void ingest(Connector connector, FileSystem fs, Opts opts, BatchWriterOpts bwOpts) throws IOException, AccumuloException, + AccumuloSecurityException, TableNotFoundException, MutationsRejectedException, TableExistsException { + long stopTime; + + byte[][] bytevals = generateValues(opts.dataSize); + + byte randomValue[] = new byte[opts.dataSize]; + Random random = new Random(); + + long bytesWritten = 0; + + createTable(connector, opts); + + BatchWriter bw = null; + FileSKVWriter writer = null; + + if (opts.outputFile != null) { + Configuration conf = CachedConfiguration.getInstance(); + writer = FileOperations.getInstance().newWriterBuilder().forFile(opts.outputFile + "." + RFile.EXTENSION, fs, conf) + .withTableConfiguration(AccumuloConfiguration.getDefaultConfiguration()).build(); + writer.startDefaultLocalityGroup(); + } else { + bw = connector.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig()); + connector.securityOperations().changeUserAuthorizations(opts.getPrincipal(), AUTHS); + } + Text labBA = new Text(opts.columnVisibility.getExpression()); + + long startTime = System.currentTimeMillis(); + for (int i = 0; i < opts.rows; i++) { + int rowid; + if (opts.stride > 0) { + rowid = ((i % opts.stride) * (opts.rows / opts.stride)) + (i / opts.stride); + } else { + rowid = i; + } + + Text row = generateRow(rowid, opts.startRow); + Mutation m = new Mutation(row); + for (int j = 0; j < opts.cols; j++) { + Text colf = new Text(opts.columnFamily); + Text colq = new Text(FastFormat.toZeroPaddedString(j, 7, 10, COL_PREFIX)); + + if (writer != null) { + Key key = new Key(row, colf, colq, labBA); + if (opts.timestamp >= 0) { + key.setTimestamp(opts.timestamp); + } else { + key.setTimestamp(startTime); + } + + if (opts.delete) { + key.setDeleted(true); + } else { + key.setDeleted(false); + } + + bytesWritten += key.getSize(); + + if (opts.delete) { + writer.append(key, new Value(new byte[0])); + } else { + byte value[]; + if (opts.random != null) { + value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j); + } else { + value = bytevals[j % bytevals.length]; + } + + Value v = new Value(value); + writer.append(key, v); + bytesWritten += v.getSize(); + } + + } else { + Key key = new Key(row, colf, colq, labBA); + bytesWritten += key.getSize(); + + if (opts.delete) { + if (opts.timestamp >= 0) + m.putDelete(colf, colq, opts.columnVisibility, opts.timestamp); + else + m.putDelete(colf, colq, opts.columnVisibility); + } else { + byte value[]; + if (opts.random != null) { + value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j); + } else { + value = bytevals[j % bytevals.length]; + } + bytesWritten += value.length; + + if (opts.timestamp >= 0) { + m.put(colf, colq, opts.columnVisibility, opts.timestamp, new Value(value, true)); + } else { + m.put(colf, colq, opts.columnVisibility, new Value(value, true)); + + } + } + } + + } + if (bw != null) + bw.addMutation(m); + + } + + if (writer != null) { + writer.close(); + } else if (bw != null) { + try { + bw.close(); + } catch (MutationsRejectedException e) { + if (e.getSecurityErrorCodes().size() > 0) { + for (Entry> entry : e.getSecurityErrorCodes().entrySet()) { + System.err.println("ERROR : Not authorized to write to : " + entry.getKey() + " due to " + entry.getValue()); + } + } + + if (e.getConstraintViolationSummaries().size() > 0) { + for (ConstraintViolationSummary cvs : e.getConstraintViolationSummaries()) { + System.err.println("ERROR : Constraint violates : " + cvs); + } + } + + throw e; + } + } + + stopTime = System.currentTimeMillis(); + + int totalValues = opts.rows * opts.cols; + double elapsed = (stopTime - startTime) / 1000.0; + + System.out.printf("%,12d records written | %,8d records/sec | %,12d bytes written | %,8d bytes/sec | %6.3f secs %n", totalValues, + (int) (totalValues / elapsed), bytesWritten, (int) (bytesWritten / elapsed), elapsed); + } + + public static void ingest(Connector c, Opts opts, BatchWriterOpts batchWriterOpts) throws MutationsRejectedException, IOException, AccumuloException, + AccumuloSecurityException, TableNotFoundException, TableExistsException { + ingest(c, FileSystem.get(CachedConfiguration.getInstance()), opts, batchWriterOpts); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/ingest/VerifyIngest.java b/core/src/main/java/org/apache/accumulo/testing/core/ingest/VerifyIngest.java new file mode 100644 index 00000000..4f8862a2 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/ingest/VerifyIngest.java @@ -0,0 +1,237 @@ +/* + * 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.testing.core.ingest; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.Random; + +import org.apache.accumulo.core.cli.ScannerOpts; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.PartialKey; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.trace.DistributedTrace; +import org.apache.accumulo.core.trace.Trace; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.beust.jcommander.Parameter; + +public class VerifyIngest { + + private static final Logger log = LoggerFactory.getLogger(VerifyIngest.class); + + public static int getRow(Key k) { + return Integer.parseInt(k.getRow().toString().split("_")[1]); + } + + public static int getCol(Key k) { + return Integer.parseInt(k.getColumnQualifier().toString().split("_")[1]); + } + + public static class Opts extends TestIngest.Opts { + @Parameter(names = "-useGet", description = "fetches values one at a time, instead of scanning") + public boolean useGet = false; + } + + public static void main(String[] args) throws Exception { + Opts opts = new Opts(); + ScannerOpts scanOpts = new ScannerOpts(); + opts.parseArgs(VerifyIngest.class.getName(), args, scanOpts); + try { + if (opts.trace) { + String name = VerifyIngest.class.getSimpleName(); + DistributedTrace.enable(); + Trace.on(name); + Trace.data("cmdLine", Arrays.asList(args).toString()); + } + + verifyIngest(opts.getConnector(), opts, scanOpts); + + } finally { + Trace.off(); + DistributedTrace.disable(); + } + } + + public static void verifyIngest(Connector connector, Opts opts, ScannerOpts scanOpts) throws AccumuloException, AccumuloSecurityException, + TableNotFoundException { + byte[][] bytevals = TestIngest.generateValues(opts.dataSize); + + Authorizations labelAuths = new Authorizations("L1", "L2", "G1", "GROUP2"); + connector.securityOperations().changeUserAuthorizations(opts.getPrincipal(), labelAuths); + + int expectedRow = opts.startRow; + int expectedCol = 0; + int recsRead = 0; + + long bytesRead = 0; + long t1 = System.currentTimeMillis(); + + byte randomValue[] = new byte[opts.dataSize]; + Random random = new Random(); + + Key endKey = new Key(new Text("row_" + String.format("%010d", opts.rows + opts.startRow))); + + int errors = 0; + + while (expectedRow < (opts.rows + opts.startRow)) { + + if (opts.useGet) { + Text rowKey = new Text("row_" + String.format("%010d", expectedRow + opts.startRow)); + Text colf = new Text(opts.columnFamily); + Text colq = new Text("col_" + String.format("%07d", expectedCol)); + + Scanner scanner = connector.createScanner("test_ingest", labelAuths); + scanner.setBatchSize(1); + Key startKey = new Key(rowKey, colf, colq); + Range range = new Range(startKey, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL)); + scanner.setRange(range); + + byte[] val = null; // t.get(rowKey, column); + + Iterator> iter = scanner.iterator(); + + if (iter.hasNext()) { + val = iter.next().getValue().get(); + } + + byte ev[]; + if (opts.random != null) { + ev = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, expectedCol); + } else { + ev = bytevals[expectedCol % bytevals.length]; + } + + if (val == null) { + log.error("Did not find " + rowKey + " " + colf + " " + colq); + errors++; + } else { + recsRead++; + bytesRead += val.length; + Value value = new Value(val); + if (value.compareTo(ev) != 0) { + log.error("unexpected value (" + rowKey + " " + colf + " " + colq + " : saw " + value + " expected " + new Value(ev)); + errors++; + } + } + + expectedCol++; + if (expectedCol >= opts.cols) { + expectedCol = 0; + expectedRow++; + } + + } else { + + Key startKey = new Key(new Text("row_" + String.format("%010d", expectedRow))); + + 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++) { + scanner.fetchColumn(new Text(opts.columnFamily), new Text("col_" + String.format("%07d", j))); + } + + int recsReadBefore = recsRead; + + for (Entry entry : scanner) { + + recsRead++; + + bytesRead += entry.getKey().getLength(); + bytesRead += entry.getValue().getSize(); + + int rowNum = getRow(entry.getKey()); + int colNum = getCol(entry.getKey()); + + if (rowNum != expectedRow) { + log.error("rowNum != expectedRow " + rowNum + " != " + expectedRow); + errors++; + expectedRow = rowNum; + } + + if (colNum != expectedCol) { + log.error("colNum != expectedCol " + colNum + " != " + expectedCol + " rowNum : " + rowNum); + errors++; + } + + if (expectedRow >= (opts.rows + opts.startRow)) { + log.error("expectedRow (" + expectedRow + ") >= (ingestArgs.rows + ingestArgs.startRow) (" + (opts.rows + opts.startRow) + + "), get batch returned data passed end key"); + errors++; + break; + } + + byte value[]; + if (opts.random != null) { + value = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, colNum); + } else { + value = bytevals[colNum % bytevals.length]; + } + + if (entry.getValue().compareTo(value) != 0) { + log.error("unexpected value, rowNum : " + rowNum + " colNum : " + colNum); + log.error(" saw = " + new String(entry.getValue().get()) + " expected = " + new String(value)); + errors++; + } + + if (opts.timestamp >= 0 && entry.getKey().getTimestamp() != opts.timestamp) { + log.error("unexpected timestamp " + entry.getKey().getTimestamp() + ", rowNum : " + rowNum + " colNum : " + colNum); + errors++; + } + + expectedCol++; + if (expectedCol >= opts.cols) { + expectedCol = 0; + expectedRow++; + } + + } + + if (recsRead == recsReadBefore) { + log.warn("Scan returned nothing, breaking..."); + break; + } + + } + } + + long t2 = System.currentTimeMillis(); + + if (errors > 0) { + throw new AccumuloException("saw " + errors + " errors "); + } + + if (expectedRow != (opts.rows + opts.startRow)) { + throw new AccumuloException("Did not read expected number of rows. Saw " + (expectedRow - opts.startRow) + " expected " + opts.rows); + } else { + System.out.printf("%,12d records read | %,8d records/sec | %,12d bytes read | %,8d bytes/sec | %6.3f secs %n", recsRead, + (int) ((recsRead) / ((t2 - t1) / 1000.0)), bytesRead, (int) (bytesRead / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0); + } + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/RowHash.java b/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/RowHash.java new file mode 100644 index 00000000..cb91e194 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/RowHash.java @@ -0,0 +1,95 @@ +/* + * 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.testing.core.mapreduce; + +import java.io.IOException; +import java.util.Base64; +import java.util.Collections; + +import org.apache.accumulo.core.cli.MapReduceClientOnRequiredTable; +import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat; +import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.util.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.MD5Hash; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +import com.beust.jcommander.Parameter; + +public class RowHash extends Configured implements Tool { + /** + * The Mapper class that given a row number, will generate the appropriate output line. + */ + public static class HashDataMapper extends Mapper { + @Override + public void map(Key row, Value data, Context context) throws IOException, InterruptedException { + Mutation m = new Mutation(row.getRow()); + m.put(new Text("cf-HASHTYPE"), new Text("cq-MD5BASE64"), new Value(Base64.getEncoder().encode(MD5Hash.digest(data.toString()).getDigest()))); + context.write(null, m); + context.progress(); + } + + @Override + public void setup(Context job) {} + } + + private static class Opts extends MapReduceClientOnRequiredTable { + @Parameter(names = "--column", required = true) + String column; + } + + @Override + public int run(String[] args) throws Exception { + Job job = Job.getInstance(getConf()); + job.setJobName(this.getClass().getName()); + job.setJarByClass(this.getClass()); + Opts opts = new Opts(); + opts.parseArgs(RowHash.class.getName(), args); + job.setInputFormatClass(AccumuloInputFormat.class); + opts.setAccumuloConfigs(job); + + String col = opts.column; + int idx = col.indexOf(":"); + Text cf = new Text(idx < 0 ? col : col.substring(0, idx)); + Text cq = idx < 0 ? null : new Text(col.substring(idx + 1)); + if (cf.getLength() > 0) + AccumuloInputFormat.fetchColumns(job, Collections.singleton(new Pair<>(cf, cq))); + + job.setMapperClass(HashDataMapper.class); + job.setMapOutputKeyClass(Text.class); + job.setMapOutputValueClass(Mutation.class); + + job.setNumReduceTasks(0); + + job.setOutputFormatClass(AccumuloOutputFormat.class); + + job.waitForCompletion(true); + return job.isSuccessful() ? 0 : 1; + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new Configuration(), new RowHash(), args); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/TeraSortIngest.java b/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/TeraSortIngest.java new file mode 100644 index 00000000..6e18000f --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/mapreduce/TeraSortIngest.java @@ -0,0 +1,399 @@ +/* + * 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.testing.core.mapreduce; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import org.apache.accumulo.core.cli.MapReduceClientOnRequiredTable; +import org.apache.accumulo.core.client.BatchWriterConfig; +import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Value; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +import com.beust.jcommander.Parameter; + +/** + * Generate the *almost* official terasort input data set. (See below) The user specifies the number of rows and the output directory and this class runs a + * map/reduce program to generate the data. The format of the data is: + *
    + *
  • (10 bytes key) (10 bytes rowid) (78 bytes filler) \r \n + *
  • The keys are random characters from the set ' ' .. '~'. + *
  • The rowid is the right justified row id as a int. + *
  • The filler consists of 7 runs of 10 characters from 'A' to 'Z'. + *
+ * + * This TeraSort is slightly modified to allow for variable length key sizes and value sizes. The row length isn't variable. To generate a terabyte of data in + * the same way TeraSort does use 10000000000 rows and 10/10 byte key length and 78/78 byte value length. Along with the 10 byte row id and \r\n this gives you + * 100 byte row * 10000000000 rows = 1tb. Min/Max ranges for key and value parameters are inclusive/inclusive respectively. + * + * + */ +public class TeraSortIngest extends Configured implements Tool { + /** + * An input format that assigns ranges of longs to each mapper. + */ + static class RangeInputFormat extends InputFormat { + /** + * An input split consisting of a range on numbers. + */ + static class RangeInputSplit extends InputSplit implements Writable { + long firstRow; + long rowCount; + + public RangeInputSplit() {} + + public RangeInputSplit(long offset, long length) { + firstRow = offset; + rowCount = length; + } + + @Override + public long getLength() throws IOException { + return 0; + } + + @Override + public String[] getLocations() throws IOException { + return new String[] {}; + } + + @Override + public void readFields(DataInput in) throws IOException { + firstRow = WritableUtils.readVLong(in); + rowCount = WritableUtils.readVLong(in); + } + + @Override + public void write(DataOutput out) throws IOException { + WritableUtils.writeVLong(out, firstRow); + WritableUtils.writeVLong(out, rowCount); + } + } + + /** + * A record reader that will generate a range of numbers. + */ + static class RangeRecordReader extends RecordReader { + long startRow; + long finishedRows; + long totalRows; + + public RangeRecordReader(RangeInputSplit split) { + startRow = split.firstRow; + finishedRows = 0; + totalRows = split.rowCount; + } + + @Override + public void close() throws IOException {} + + @Override + public float getProgress() throws IOException { + return finishedRows / (float) totalRows; + } + + @Override + public LongWritable getCurrentKey() throws IOException, InterruptedException { + return new LongWritable(startRow + finishedRows); + } + + @Override + public NullWritable getCurrentValue() throws IOException, InterruptedException { + return NullWritable.get(); + } + + @Override + public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {} + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + if (finishedRows < totalRows) { + ++finishedRows; + return true; + } + return false; + } + } + + @Override + public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException { + // reporter.setStatus("Creating record reader"); + return new RangeRecordReader((RangeInputSplit) split); + } + + /** + * Create the desired number of splits, dividing the number of rows between the mappers. + */ + @Override + public List getSplits(JobContext job) { + long totalRows = job.getConfiguration().getLong(NUMROWS, 0); + int numSplits = job.getConfiguration().getInt(NUMSPLITS, 1); + long rowsPerSplit = totalRows / numSplits; + System.out.println("Generating " + totalRows + " using " + numSplits + " maps with step of " + rowsPerSplit); + ArrayList splits = new ArrayList<>(numSplits); + long currentRow = 0; + for (int split = 0; split < numSplits - 1; ++split) { + splits.add(new RangeInputSplit(currentRow, rowsPerSplit)); + currentRow += rowsPerSplit; + } + splits.add(new RangeInputSplit(currentRow, totalRows - currentRow)); + System.out.println("Done Generating."); + return splits; + } + + } + + private static String NUMSPLITS = "terasort.overridesplits"; + private static String NUMROWS = "terasort.numrows"; + + static class RandomGenerator { + private long seed = 0; + private static final long mask32 = (1l << 32) - 1; + /** + * The number of iterations separating the precomputed seeds. + */ + private static final int seedSkip = 128 * 1024 * 1024; + /** + * The precomputed seed values after every seedSkip iterations. There should be enough values so that a 2**32 iterations are covered. + */ + private static final long[] seeds = new long[] {0L, 4160749568L, 4026531840L, 3892314112L, 3758096384L, 3623878656L, 3489660928L, 3355443200L, 3221225472L, + 3087007744L, 2952790016L, 2818572288L, 2684354560L, 2550136832L, 2415919104L, 2281701376L, 2147483648L, 2013265920L, 1879048192L, 1744830464L, + 1610612736L, 1476395008L, 1342177280L, 1207959552L, 1073741824L, 939524096L, 805306368L, 671088640L, 536870912L, 402653184L, 268435456L, 134217728L,}; + + /** + * Start the random number generator on the given iteration. + * + * @param initalIteration + * the iteration number to start on + */ + RandomGenerator(long initalIteration) { + int baseIndex = (int) ((initalIteration & mask32) / seedSkip); + seed = seeds[baseIndex]; + for (int i = 0; i < initalIteration % seedSkip; ++i) { + next(); + } + } + + RandomGenerator() { + this(0); + } + + long next() { + seed = (seed * 3141592621l + 663896637) & mask32; + return seed; + } + } + + /** + * The Mapper class that given a row number, will generate the appropriate output line. + */ + public static class SortGenMapper extends Mapper { + private Text tableName = null; + private int minkeylength = 0; + private int maxkeylength = 0; + private int minvaluelength = 0; + private int maxvaluelength = 0; + + private Text key = new Text(); + private Text value = new Text(); + private RandomGenerator rand; + private byte[] keyBytes; // = new byte[12]; + private byte[] spaces = " ".getBytes(); + private byte[][] filler = new byte[26][]; + { + for (int i = 0; i < 26; ++i) { + filler[i] = new byte[10]; + for (int j = 0; j < 10; ++j) { + filler[i][j] = (byte) ('A' + i); + } + } + } + + /** + * Add a random key to the text + */ + private Random random = new Random(); + + private void addKey() { + int range = random.nextInt(maxkeylength - minkeylength + 1); + int keylen = range + minkeylength; + int keyceil = keylen + (4 - (keylen % 4)); + keyBytes = new byte[keyceil]; + + long temp = 0; + for (int i = 0; i < keyceil / 4; i++) { + temp = rand.next() / 52; + keyBytes[3 + 4 * i] = (byte) (' ' + (temp % 95)); + temp /= 95; + keyBytes[2 + 4 * i] = (byte) (' ' + (temp % 95)); + temp /= 95; + keyBytes[1 + 4 * i] = (byte) (' ' + (temp % 95)); + temp /= 95; + keyBytes[4 * i] = (byte) (' ' + (temp % 95)); + } + key.set(keyBytes, 0, keylen); + } + + /** + * Add the rowid to the row. + */ + private Text getRowIdString(long rowId) { + Text paddedRowIdString = new Text(); + byte[] rowid = Integer.toString((int) rowId).getBytes(); + int padSpace = 10 - rowid.length; + if (padSpace > 0) { + paddedRowIdString.append(spaces, 0, 10 - rowid.length); + } + paddedRowIdString.append(rowid, 0, Math.min(rowid.length, 10)); + return paddedRowIdString; + } + + /** + * Add the required filler bytes. Each row consists of 7 blocks of 10 characters and 1 block of 8 characters. + * + * @param rowId + * the current row number + */ + private void addFiller(long rowId) { + int base = (int) ((rowId * 8) % 26); + + // Get Random var + Random random = new Random(rand.seed); + + int range = random.nextInt(maxvaluelength - minvaluelength + 1); + int valuelen = range + minvaluelength; + + while (valuelen > 10) { + value.append(filler[(base + valuelen) % 26], 0, 10); + valuelen -= 10; + } + + if (valuelen > 0) + value.append(filler[(base + valuelen) % 26], 0, valuelen); + } + + @Override + public void map(LongWritable row, NullWritable ignored, Context context) throws IOException, InterruptedException { + context.setStatus("Entering"); + long rowId = row.get(); + if (rand == null) { + // we use 3 random numbers per a row + rand = new RandomGenerator(rowId * 3); + } + addKey(); + value.clear(); + // addRowId(rowId); + addFiller(rowId); + + // New + Mutation m = new Mutation(key); + m.put(new Text("c"), // column family + getRowIdString(rowId), // column qual + new Value(value.toString().getBytes())); // data + + context.setStatus("About to add to accumulo"); + context.write(tableName, m); + context.setStatus("Added to accumulo " + key.toString()); + } + + @Override + public void setup(Context job) { + minkeylength = job.getConfiguration().getInt("cloudgen.minkeylength", 0); + maxkeylength = job.getConfiguration().getInt("cloudgen.maxkeylength", 0); + minvaluelength = job.getConfiguration().getInt("cloudgen.minvaluelength", 0); + maxvaluelength = job.getConfiguration().getInt("cloudgen.maxvaluelength", 0); + tableName = new Text(job.getConfiguration().get("cloudgen.tablename")); + } + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new Configuration(), new TeraSortIngest(), args); + } + + static class Opts extends MapReduceClientOnRequiredTable { + @Parameter(names = "--count", description = "number of rows to ingest", required = true) + long numRows; + @Parameter(names = {"-nk", "--minKeySize"}, description = "miniumum key size", required = true) + int minKeyLength; + @Parameter(names = {"-xk", "--maxKeySize"}, description = "maximum key size", required = true) + int maxKeyLength; + @Parameter(names = {"-nv", "--minValueSize"}, description = "minimum key size", required = true) + int minValueLength; + @Parameter(names = {"-xv", "--maxValueSize"}, description = "maximum key size", required = true) + int maxValueLength; + @Parameter(names = "--splits", description = "number of splits to create in the table") + int splits = 0; + } + + @Override + public int run(String[] args) throws Exception { + Job job = Job.getInstance(getConf()); + job.setJobName("TeraSortCloud"); + job.setJarByClass(this.getClass()); + Opts opts = new Opts(); + opts.parseArgs(TeraSortIngest.class.getName(), args); + + job.setInputFormatClass(RangeInputFormat.class); + job.setMapperClass(SortGenMapper.class); + job.setMapOutputKeyClass(Text.class); + job.setMapOutputValueClass(Mutation.class); + + job.setNumReduceTasks(0); + + job.setOutputFormatClass(AccumuloOutputFormat.class); + opts.setAccumuloConfigs(job); + BatchWriterConfig bwConfig = new BatchWriterConfig().setMaxMemory(10L * 1000 * 1000); + AccumuloOutputFormat.setBatchWriterOptions(job, bwConfig); + + Configuration conf = job.getConfiguration(); + conf.setLong(NUMROWS, opts.numRows); + conf.setInt("cloudgen.minkeylength", opts.minKeyLength); + conf.setInt("cloudgen.maxkeylength", opts.maxKeyLength); + conf.setInt("cloudgen.minvaluelength", opts.minValueLength); + conf.setInt("cloudgen.maxvaluelength", opts.maxValueLength); + conf.set("cloudgen.tablename", opts.getTableName()); + + if (args.length > 10) + conf.setInt(NUMSPLITS, opts.splits); + + job.waitForCompletion(true); + return job.isSuccessful() ? 0 : 1; + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTree.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTree.java new file mode 100644 index 00000000..003cd5de --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTree.java @@ -0,0 +1,92 @@ +/* + * 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.testing.core.merkle; + +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.accumulo.core.util.Pair; + +import com.google.common.collect.Iterables; + +/** + * Simple implementation of a Merkle tree + */ +public class MerkleTree { + protected List leaves; + protected String digestAlgorithm; + + public MerkleTree(List leaves, String digestAlgorithm) { + this.leaves = leaves; + this.digestAlgorithm = digestAlgorithm; + } + + public MerkleTreeNode getRootNode() throws NoSuchAlgorithmException { + ArrayList buffer = new ArrayList<>(leaves.size()); + buffer.addAll(leaves); + + while (buffer.size() > 1) { + // Find two nodes that we want to roll up + Pair pairToJoin = findNextPair(buffer); + + // Make a parent node from them + MerkleTreeNode parent = new MerkleTreeNode(Arrays.asList(buffer.get(pairToJoin.getFirst()), buffer.get(pairToJoin.getSecond())), digestAlgorithm); + + // Insert it back into the "tree" at the position of the first child + buffer.set(pairToJoin.getFirst(), parent); + + // Remove the second child completely + buffer.remove(pairToJoin.getSecond().intValue()); + + // "recurse" + } + + return Iterables.getOnlyElement(buffer); + } + + protected Pair findNextPair(List nodes) { + int i = 0, j = 1; + while (i < nodes.size() && j < nodes.size()) { + MerkleTreeNode left = nodes.get(i), right = nodes.get(j); + + // At the same level + if (left.getLevel() == right.getLevel()) { + return new Pair<>(i, j); + } + + // Peek to see if we have another element + if (j + 1 < nodes.size()) { + // If we do, try to match those + i++; + j++; + } else { + // Otherwise, the last two elements must be paired + return new Pair<>(i, j); + } + } + + if (2 < nodes.size()) { + throw new IllegalStateException("Should not have exited loop without pairing two elements when we have at least 3 nodes"); + } else if (2 == nodes.size()) { + return new Pair<>(0, 1); + } else { + throw new IllegalStateException("Must have at least two nodes to pair"); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTreeNode.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTreeNode.java new file mode 100644 index 00000000..168b6e1b --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/MerkleTreeNode.java @@ -0,0 +1,131 @@ +/* + * 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.testing.core.merkle; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encapsulates the level (height) within the tree, the ranges that it covers, and the new hash + */ +public class MerkleTreeNode { + private static final Logger log = LoggerFactory.getLogger(MerkleTreeNode.class); + + private Range range; + private int level; + private List children; + private byte[] hash; + + public MerkleTreeNode(Range range, int level, List children, byte[] hash) { + this.range = range; + this.level = level; + this.children = children; + this.hash = hash; + } + + public MerkleTreeNode(Key k, Value v) { + range = RangeSerialization.toRange(k); + level = 0; + children = Collections.emptyList(); + hash = v.get(); + } + + public MerkleTreeNode(List children, String digestAlgorithm) throws NoSuchAlgorithmException { + level = 0; + this.children = new ArrayList<>(children.size()); + MessageDigest digest = MessageDigest.getInstance(digestAlgorithm); + + Range childrenRange = null; + for (MerkleTreeNode child : children) { + this.children.add(child.getRange()); + level = Math.max(child.getLevel(), level); + digest.update(child.getHash()); + + if (null == childrenRange) { + childrenRange = child.getRange(); + } else { + List overlappingRanges = Range.mergeOverlapping(Arrays.asList(childrenRange, child.getRange())); + if (1 != overlappingRanges.size()) { + log.error("Tried to merge non-contiguous ranges: {} {}", childrenRange, child.getRange()); + throw new IllegalArgumentException("Ranges must be contiguous: " + childrenRange + ", " + child.getRange()); + } + + childrenRange = overlappingRanges.get(0); + } + } + + // Our actual level is one more than the highest level of our children + level++; + + // Roll the hash up the tree + hash = digest.digest(); + + // Set the range to be the merged result of the children + range = childrenRange; + } + + public Range getRange() { + return range; + } + + public int getLevel() { + return level; + } + + public List getChildren() { + return children; + } + + public byte[] getHash() { + return hash; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(32); + sb.append("range=").append(range).append(" level=").append(level).append(" hash=").append(Hex.encodeHexString(hash)).append(" children=").append(children); + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (o instanceof MerkleTreeNode) { + MerkleTreeNode other = (MerkleTreeNode) o; + return range.equals(other.getRange()) && level == other.getLevel() && children.equals(other.getChildren()) && Arrays.equals(hash, other.getHash()); + } + + return false; + } + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(1395, 39532); + return hcb.append(range).append(level).append(children).append(hash).toHashCode(); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/RangeSerialization.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/RangeSerialization.java new file mode 100644 index 00000000..c301d494 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/RangeSerialization.java @@ -0,0 +1,72 @@ +/* + * 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.testing.core.merkle; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.hadoop.io.Text; + +/** + * + */ +public class RangeSerialization { + private static final Text EMPTY = new Text(new byte[0]); + + public static Range toRange(Key key) { + Text holder = new Text(); + key.getRow(holder); + Key startKey; + if (0 == holder.getLength()) { + startKey = null; + } else { + startKey = new Key(holder); + } + + key.getColumnQualifier(holder); + Key endKey; + if (0 == holder.getLength()) { + endKey = null; + } else { + endKey = new Key(holder); + } + + // Don't be inclusive for no bounds on a Range + return new Range(startKey, startKey != null, endKey, endKey != null); + } + + public static Key toKey(Range range) { + Text row = getRow(range); + return new Key(row, EMPTY, getColumnQualifier(range)); + } + + public static Mutation toMutation(Range range, Value v) { + Text row = getRow(range); + Mutation m = new Mutation(row); + m.put(EMPTY, getColumnQualifier(range), v); + return m; + } + + public static Text getRow(Range range) { + return range.isInfiniteStartKey() ? EMPTY : range.getStartKey().getRow(); + } + + public static Text getColumnQualifier(Range range) { + return range.isInfiniteStopKey() ? EMPTY : range.getEndKey().getRow(); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/CompareTables.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/CompareTables.java new file mode 100644 index 00000000..fd6251f1 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/CompareTables.java @@ -0,0 +1,176 @@ +/* + * 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.testing.core.merkle.cli; + +import java.io.FileNotFoundException; +import java.security.NoSuchAlgorithmException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.accumulo.core.cli.BatchWriterOpts; +import org.apache.accumulo.core.cli.ClientOpts; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.TableExistsException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.data.Range; +import org.apache.commons.codec.binary.Hex; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.beust.jcommander.Parameter; + +/** + * Accepts a set of tables, computes the hashes for each, and prints the top-level hash for each table. + *

+ * Will automatically create output tables for intermediate hashes instead of requiring their existence. This will raise an exception when the table we want to + * use already exists. + */ +public class CompareTables { + private static final Logger log = LoggerFactory.getLogger(CompareTables.class); + + public static class CompareTablesOpts extends ClientOpts { + @Parameter(names = {"--tables"}, description = "Tables to compare", variableArity = true) + public List tables; + + @Parameter(names = {"-nt", "--numThreads"}, required = false, description = "number of concurrent threads calculating digests") + private int numThreads = 4; + + @Parameter(names = {"-hash", "--hash"}, required = true, description = "type of hash to use") + private String hashName; + + @Parameter(names = {"-iter", "--iterator"}, required = false, description = "Should pushdown digest to iterators") + private boolean iteratorPushdown = false; + + @Parameter(names = {"-s", "--splits"}, required = false, description = "File of splits to use for merkle tree") + private String splitsFile = null; + + public List getTables() { + return this.tables; + } + + public void setTables(List tables) { + this.tables = tables; + } + + public int getNumThreads() { + return numThreads; + } + + public void setNumThreads(int numThreads) { + this.numThreads = numThreads; + } + + public String getHashName() { + return hashName; + } + + public void setHashName(String hashName) { + this.hashName = hashName; + } + + public boolean isIteratorPushdown() { + return iteratorPushdown; + } + + public void setIteratorPushdown(boolean iteratorPushdown) { + this.iteratorPushdown = iteratorPushdown; + } + + public String getSplitsFile() { + return splitsFile; + } + + public void setSplitsFile(String splitsFile) { + this.splitsFile = splitsFile; + } + } + + private CompareTablesOpts opts; + + protected CompareTables() {} + + public CompareTables(CompareTablesOpts opts) { + this.opts = opts; + } + + public Map computeAllHashes() throws AccumuloException, AccumuloSecurityException, TableExistsException, NoSuchAlgorithmException, + TableNotFoundException, FileNotFoundException { + final Connector conn = opts.getConnector(); + final Map hashesByTable = new HashMap<>(); + + for (String table : opts.getTables()) { + final String outputTableName = table + "_merkle"; + + if (conn.tableOperations().exists(outputTableName)) { + throw new IllegalArgumentException("Expected output table name to not yet exist: " + outputTableName); + } + + conn.tableOperations().create(outputTableName); + + GenerateHashes genHashes = new GenerateHashes(); + Collection ranges = genHashes.getRanges(opts.getConnector(), table, opts.getSplitsFile()); + + try { + genHashes.run(opts.getConnector(), table, table + "_merkle", opts.getHashName(), opts.getNumThreads(), opts.isIteratorPushdown(), ranges); + } catch (Exception e) { + log.error("Error generating hashes for {}", table, e); + throw new RuntimeException(e); + } + + ComputeRootHash computeRootHash = new ComputeRootHash(); + String hash = Hex.encodeHexString(computeRootHash.getHash(conn, outputTableName, opts.getHashName())); + + hashesByTable.put(table, hash); + } + + return hashesByTable; + } + + public static void main(String[] args) throws Exception { + CompareTablesOpts opts = new CompareTablesOpts(); + BatchWriterOpts bwOpts = new BatchWriterOpts(); + opts.parseArgs("CompareTables", args, bwOpts); + + if (opts.isIteratorPushdown() && null != opts.getSplitsFile()) { + throw new IllegalArgumentException("Cannot use iterator pushdown with anything other than table split points"); + } + + CompareTables compareTables = new CompareTables(opts); + Map tableToHashes = compareTables.computeAllHashes(); + + boolean hashesEqual = true; + String previousHash = null; + for (Entry entry : tableToHashes.entrySet()) { + // Set the previous hash if we dont' have one + if (null == previousHash) { + previousHash = entry.getValue(); + } else if (hashesEqual) { + // If the hashes are still equal, check that the new hash is also equal + hashesEqual = previousHash.equals(entry.getValue()); + } + + System.out.println(entry.getKey() + " " + entry.getValue()); + } + + System.exit(hashesEqual ? 0 : 1); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ComputeRootHash.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ComputeRootHash.java new file mode 100644 index 00000000..45114468 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ComputeRootHash.java @@ -0,0 +1,100 @@ +/* + * 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.testing.core.merkle.cli; + +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map.Entry; + +import org.apache.accumulo.core.cli.ClientOnRequiredTable; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.test.replication.merkle.MerkleTree; +import org.apache.accumulo.test.replication.merkle.MerkleTreeNode; +import org.apache.accumulo.test.replication.merkle.RangeSerialization; +import org.apache.commons.codec.binary.Hex; + +import com.beust.jcommander.Parameter; + +/** + * Given a table created by {@link GenerateHashes} which contains the leaves of a Merkle tree, compute the root node of the Merkle tree which can be quickly + * compared to the root node of another Merkle tree to ascertain equality. + */ +public class ComputeRootHash { + + public static class ComputeRootHashOpts extends ClientOnRequiredTable { + @Parameter(names = {"-hash", "--hash"}, required = true, description = "type of hash to use") + private String hashName; + + public String getHashName() { + return hashName; + } + + public void setHashName(String hashName) { + this.hashName = hashName; + } + } + + public byte[] getHash(ComputeRootHashOpts opts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, NoSuchAlgorithmException { + Connector conn = opts.getConnector(); + String table = opts.getTableName(); + + return getHash(conn, table, opts.getHashName()); + } + + public byte[] getHash(Connector conn, String table, String hashName) throws TableNotFoundException, NoSuchAlgorithmException { + List leaves = getLeaves(conn, table); + + MerkleTree tree = new MerkleTree(leaves, hashName); + + return tree.getRootNode().getHash(); + } + + protected ArrayList getLeaves(Connector conn, String tableName) throws TableNotFoundException { + // TODO make this a bit more resilient to very large merkle trees by lazily reading more data from the table when necessary + final Scanner s = conn.createScanner(tableName, Authorizations.EMPTY); + final ArrayList leaves = new ArrayList<>(); + + for (Entry entry : s) { + Range range = RangeSerialization.toRange(entry.getKey()); + byte[] hash = entry.getValue().get(); + + leaves.add(new MerkleTreeNode(range, 0, Collections. emptyList(), hash)); + } + + return leaves; + } + + public static void main(String[] args) throws Exception { + ComputeRootHashOpts opts = new ComputeRootHashOpts(); + opts.parseArgs("ComputeRootHash", args); + + ComputeRootHash computeRootHash = new ComputeRootHash(); + byte[] rootHash = computeRootHash.getHash(opts); + + System.out.println(Hex.encodeHexString(rootHash)); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/GenerateHashes.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/GenerateHashes.java new file mode 100644 index 00000000..35bf6848 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/GenerateHashes.java @@ -0,0 +1,287 @@ +/* + * 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.testing.core.merkle.cli; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileNotFoundException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map.Entry; +import java.util.TreeSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.apache.accumulo.core.cli.BatchWriterOpts; +import org.apache.accumulo.core.cli.ClientOnRequiredTable; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.BatchWriterConfig; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.MutationsRejectedException; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.test.replication.merkle.RangeSerialization; +import org.apache.accumulo.test.replication.merkle.skvi.DigestIterator; +import org.apache.commons.codec.binary.Hex; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.beust.jcommander.Parameter; +import com.google.common.collect.Iterables; + +/** + * Read from a table, compute a Merkle tree and output it to a table. Each key-value pair in the destination table is a leaf node of the Merkle tree. + */ +public class GenerateHashes { + private static final Logger log = LoggerFactory.getLogger(GenerateHashes.class); + + public static class GenerateHashesOpts extends ClientOnRequiredTable { + @Parameter(names = {"-hash", "--hash"}, required = true, description = "type of hash to use") + private String hashName; + + @Parameter(names = {"-o", "--output"}, required = true, description = "output table name, expected to exist and be writable") + private String outputTableName; + + @Parameter(names = {"-nt", "--numThreads"}, required = false, description = "number of concurrent threads calculating digests") + private int numThreads = 4; + + @Parameter(names = {"-iter", "--iterator"}, required = false, description = "Should we push down logic with an iterator") + private boolean iteratorPushdown = false; + + @Parameter(names = {"-s", "--splits"}, required = false, description = "File of splits to use for merkle tree") + private String splitsFile = null; + + public String getHashName() { + return hashName; + } + + public void setHashName(String hashName) { + this.hashName = hashName; + } + + public String getOutputTableName() { + return outputTableName; + } + + public void setOutputTableName(String outputTableName) { + this.outputTableName = outputTableName; + } + + public int getNumThreads() { + return numThreads; + } + + public void setNumThreads(int numThreads) { + this.numThreads = numThreads; + } + + public boolean isIteratorPushdown() { + return iteratorPushdown; + } + + public void setIteratorPushdown(boolean iteratorPushdown) { + this.iteratorPushdown = iteratorPushdown; + } + + public String getSplitsFile() { + return splitsFile; + } + + public void setSplitsFile(String splitsFile) { + this.splitsFile = splitsFile; + } + } + + public Collection getRanges(Connector conn, String tableName, String splitsFile) throws TableNotFoundException, AccumuloSecurityException, + AccumuloException, FileNotFoundException { + if (null == splitsFile) { + log.info("Using table split points"); + Collection endRows = conn.tableOperations().listSplits(tableName); + return endRowsToRanges(endRows); + } else { + log.info("Using provided split points"); + ArrayList splits = new ArrayList<>(); + + String line; + java.util.Scanner file = new java.util.Scanner(new File(splitsFile), UTF_8.name()); + try { + while (file.hasNextLine()) { + line = file.nextLine(); + if (!line.isEmpty()) { + splits.add(new Text(line)); + } + } + } finally { + file.close(); + } + + Collections.sort(splits); + return endRowsToRanges(splits); + } + } + + public void run(GenerateHashesOpts opts) throws TableNotFoundException, AccumuloSecurityException, AccumuloException, NoSuchAlgorithmException, + FileNotFoundException { + Collection ranges = getRanges(opts.getConnector(), opts.getTableName(), opts.getSplitsFile()); + + run(opts.getConnector(), opts.getTableName(), opts.getOutputTableName(), opts.getHashName(), opts.getNumThreads(), opts.isIteratorPushdown(), ranges); + } + + public void run(final Connector conn, final String inputTableName, final String outputTableName, final String digestName, int numThreads, + final boolean iteratorPushdown, final Collection ranges) throws TableNotFoundException, AccumuloSecurityException, AccumuloException, + NoSuchAlgorithmException { + if (!conn.tableOperations().exists(outputTableName)) { + throw new IllegalArgumentException(outputTableName + " does not exist, please create it"); + } + + // Get some parallelism + ExecutorService svc = Executors.newFixedThreadPool(numThreads); + final BatchWriter bw = conn.createBatchWriter(outputTableName, new BatchWriterConfig()); + + try { + for (final Range range : ranges) { + final MessageDigest digest = getDigestAlgorithm(digestName); + + svc.execute(new Runnable() { + + @Override + public void run() { + Scanner s; + try { + s = conn.createScanner(inputTableName, Authorizations.EMPTY); + } catch (Exception e) { + log.error("Could not get scanner for " + inputTableName, e); + throw new RuntimeException(e); + } + + s.setRange(range); + + Value v = null; + Mutation m = null; + if (iteratorPushdown) { + IteratorSetting cfg = new IteratorSetting(50, DigestIterator.class); + cfg.addOption(DigestIterator.HASH_NAME_KEY, digestName); + s.addScanIterator(cfg); + + // The scanner should only ever return us one Key-Value, otherwise this approach won't work + Entry entry = Iterables.getOnlyElement(s); + + v = entry.getValue(); + m = RangeSerialization.toMutation(range, v); + } else { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + for (Entry entry : s) { + DataOutputStream out = new DataOutputStream(baos); + try { + entry.getKey().write(out); + entry.getValue().write(out); + } catch (Exception e) { + log.error("Error writing {}", entry, e); + throw new RuntimeException(e); + } + + digest.update(baos.toByteArray()); + baos.reset(); + } + + v = new Value(digest.digest()); + m = RangeSerialization.toMutation(range, v); + } + + // Log some progress + log.info("{} computed digest for {} of {}", Thread.currentThread().getName(), range, Hex.encodeHexString(v.get())); + + try { + bw.addMutation(m); + } catch (MutationsRejectedException e) { + log.error("Could not write mutation", e); + throw new RuntimeException(e); + } + } + }); + } + + svc.shutdown(); + + // Wait indefinitely for the scans to complete + while (!svc.isTerminated()) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + log.error("Interrupted while waiting for executor service to gracefully complete. Exiting now"); + svc.shutdownNow(); + return; + } + } + } finally { + // We can only safely close this when we're exiting or we've completely all tasks + bw.close(); + } + } + + public TreeSet endRowsToRanges(Collection endRows) { + ArrayList sortedEndRows = new ArrayList<>(endRows); + Collections.sort(sortedEndRows); + + Text prevEndRow = null; + TreeSet ranges = new TreeSet<>(); + for (Text endRow : sortedEndRows) { + if (null == prevEndRow) { + ranges.add(new Range(null, false, endRow, true)); + } else { + ranges.add(new Range(prevEndRow, false, endRow, true)); + } + prevEndRow = endRow; + } + + ranges.add(new Range(prevEndRow, false, null, false)); + + return ranges; + } + + protected MessageDigest getDigestAlgorithm(String digestName) throws NoSuchAlgorithmException { + return MessageDigest.getInstance(digestName); + } + + public static void main(String[] args) throws Exception { + GenerateHashesOpts opts = new GenerateHashesOpts(); + BatchWriterOpts bwOpts = new BatchWriterOpts(); + opts.parseArgs(GenerateHashes.class.getName(), args, bwOpts); + + if (opts.isIteratorPushdown() && null != opts.getSplitsFile()) { + throw new IllegalArgumentException("Cannot use iterator pushdown with anything other than table split points"); + } + + GenerateHashes generate = new GenerateHashes(); + generate.run(opts); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ManualComparison.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ManualComparison.java new file mode 100644 index 00000000..87489f62 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/cli/ManualComparison.java @@ -0,0 +1,95 @@ +/* + * 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.testing.core.merkle.cli; + +import java.util.Iterator; +import java.util.Map.Entry; + +import org.apache.accumulo.core.cli.ClientOpts; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.security.Authorizations; + +import com.beust.jcommander.Parameter; + +/** + * Accepts two table names and enumerates all key-values pairs in both checking for correctness. All differences between the two tables will be printed to the + * console. + */ +public class ManualComparison { + + public static class ManualComparisonOpts extends ClientOpts { + @Parameter(names = {"--table1"}, required = true, description = "First table") + public String table1; + + @Parameter(names = {"--table2"}, required = true, description = "First table") + public String table2; + } + + public static void main(String[] args) throws Exception { + ManualComparisonOpts opts = new ManualComparisonOpts(); + opts.parseArgs("ManualComparison", args); + + Connector conn = opts.getConnector(); + + Scanner s1 = conn.createScanner(opts.table1, Authorizations.EMPTY), s2 = conn.createScanner(opts.table2, Authorizations.EMPTY); + Iterator> iter1 = s1.iterator(), iter2 = s2.iterator(); + boolean incrementFirst = true, incrementSecond = true; + + Entry entry1 = iter1.next(), entry2 = iter2.next(); + while (iter1.hasNext() && iter2.hasNext()) { + if (incrementFirst) { + entry1 = iter1.next(); + } + if (incrementSecond) { + entry2 = iter2.next(); + } + incrementFirst = false; + incrementSecond = false; + + if (!entry1.equals(entry2)) { + + if (entry1.getKey().compareTo(entry2.getKey()) < 0) { + System.out.println("Exist in original " + entry1); + incrementFirst = true; + } else if (entry2.getKey().compareTo(entry1.getKey()) < 0) { + System.out.println("Exist in replica " + entry2); + incrementSecond = true; + } else { + System.out.println("Differ... " + entry1 + " " + entry2); + incrementFirst = true; + incrementSecond = true; + } + } else { + incrementFirst = true; + incrementSecond = true; + } + } + + System.out.println("\nExtra entries from " + opts.table1); + while (iter1.hasNext()) { + System.out.println(iter1.next()); + } + + System.out.println("\nExtra entries from " + opts.table2); + while (iter2.hasNext()) { + System.out.println(iter2.next()); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/ingest/RandomWorkload.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/ingest/RandomWorkload.java new file mode 100644 index 00000000..440b0093 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/ingest/RandomWorkload.java @@ -0,0 +1,120 @@ +/* + * 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.testing.core.merkle.ingest; + +import java.util.Random; + +import org.apache.accumulo.core.cli.BatchWriterOpts; +import org.apache.accumulo.core.cli.ClientOnDefaultTable; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.BatchWriterConfig; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Value; +import org.apache.hadoop.io.Text; + +import com.beust.jcommander.Parameter; + +/** + * Generates some random data with a given percent of updates to be deletes. + */ +public class RandomWorkload { + public static final String DEFAULT_TABLE_NAME = "randomWorkload"; + + public static class RandomWorkloadOpts extends ClientOnDefaultTable { + @Parameter(names = {"-n", "--num"}, required = true, description = "Num records to write") + public long numRecords; + + @Parameter(names = {"-r", "--rows"}, required = true, description = "Range of rows that can be generated") + public int rowMax; + + @Parameter(names = {"-cf", "--colfams"}, required = true, description = "Range of column families that can be generated") + public int cfMax; + + @Parameter(names = {"-cq", "--colquals"}, required = true, description = "Range of column qualifiers that can be generated") + public int cqMax; + + @Parameter(names = {"-d", "--deletes"}, required = false, description = "Percentage of updates that should be deletes") + public int deletePercent = 5; + + public RandomWorkloadOpts() { + super(DEFAULT_TABLE_NAME); + } + + public RandomWorkloadOpts(String tableName) { + super(tableName); + } + } + + public void run(RandomWorkloadOpts opts, BatchWriterConfig cfg) throws Exception { + run(opts.getConnector(), opts.getTableName(), cfg, opts.numRecords, opts.rowMax, opts.cfMax, opts.cqMax, opts.deletePercent); + } + + public void run(final Connector conn, final String tableName, final BatchWriterConfig cfg, final long numRecords, int rowMax, int cfMax, int cqMax, + int deletePercent) throws Exception { + + final Random rowRand = new Random(12345); + final Random cfRand = new Random(12346); + final Random cqRand = new Random(12347); + final Random deleteRand = new Random(12348); + long valueCounter = 0l; + + if (!conn.tableOperations().exists(tableName)) { + conn.tableOperations().create(tableName); + } + + BatchWriter bw = conn.createBatchWriter(tableName, cfg); + try { + final Text row = new Text(), cf = new Text(), cq = new Text(); + final Value value = new Value(); + for (long i = 0; i < numRecords; i++) { + row.set(Integer.toString(rowRand.nextInt(rowMax))); + cf.set(Integer.toString(cfRand.nextInt(cfMax))); + cq.set(Integer.toString(cqRand.nextInt(cqMax))); + + Mutation m = new Mutation(row); + + // Choose a random value between [0,100) + int deleteValue = deleteRand.nextInt(100); + + // putDelete if the value we chose is less than our delete percentage + if (deleteValue < deletePercent) { + m.putDelete(cf, cq); + } else { + value.set(Long.toString(valueCounter).getBytes()); + m.put(cf, cq, valueCounter, value); + } + + bw.addMutation(m); + + valueCounter++; + } + } finally { + bw.close(); + } + } + + public static void main(String[] args) throws Exception { + RandomWorkloadOpts opts = new RandomWorkloadOpts(); + BatchWriterOpts bwOpts = new BatchWriterOpts(); + opts.parseArgs(RandomWorkload.class.getSimpleName(), args, bwOpts); + + RandomWorkload rw = new RandomWorkload(); + + rw.run(opts, bwOpts.getBatchWriterConfig()); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/package-info.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/package-info.java new file mode 100644 index 00000000..5b27e4bb --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/package-info.java @@ -0,0 +1,39 @@ +/* + * 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. + */ +/** + * A Merkle tree is a hash tree and can be used to evaluate equality over large + * files with the ability to ascertain what portions of the files differ. Each leaf of the Merkle tree is some hash of a + * portion of the file, with each leaf corresponding to some "range" within the source file. As such, if all leaves are + * considered as ranges of the source file, the "sum" of all leaves creates a contiguous range over the entire file. + *

+ * The parent of any nodes (typically, a binary tree; however this is not required) is the concatenation of the hashes of + * the children. We can construct a full tree by walking up the tree, creating parents from children, until we have a root + * node. To check equality of two files that each have a merkle tree built, we can very easily compare the value of at the + * root of the Merkle tree to know whether or not the files are the same. + *

+ * Additionally, in the situation where we have two files with we expect to be the same but are not, we can walk back down + * the tree, finding subtrees that are equal and subtrees that are not. Subtrees that are equal correspond to portions of + * the files which are identical, where subtrees that are not equal correspond to discrepancies between the two files. + *

+ * We can apply this concept to Accumulo, treating a table as a file, and ranges within a file as an Accumulo Range. We can + * then compute the hashes over each of these Ranges and compute the entire Merkle tree to determine if two tables are + * equivalent. + * + * @since 1.7.0 + */ +package org.apache.accumulo.testing.core.merkle; + diff --git a/core/src/main/java/org/apache/accumulo/testing/core/merkle/skvi/DigestIterator.java b/core/src/main/java/org/apache/accumulo/testing/core/merkle/skvi/DigestIterator.java new file mode 100644 index 00000000..ab7ad433 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/merkle/skvi/DigestIterator.java @@ -0,0 +1,149 @@ +/* + * 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.testin.core.merkle.skvi; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Collection; +import java.util.Map; + +import org.apache.accumulo.core.data.ByteSequence; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; + +/** + * {@link SortedKeyValueIterator} which attempts to compute a hash over some range of Key-Value pairs. + *

+ * For the purposes of constructing a Merkle tree, this class will only generate a meaningful result if the (Batch)Scanner will compute a single digest over a + * Range. If the (Batch)Scanner stops and restarts in the middle of a session, incorrect values will be returned and the merkle tree will be invalid. + */ +public class DigestIterator implements SortedKeyValueIterator { + public static final String HASH_NAME_KEY = "hash.name"; + + private MessageDigest digest; + private Key topKey; + private Value topValue; + private SortedKeyValueIterator source; + + @Override + public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { + String hashName = options.get(HASH_NAME_KEY); + if (null == hashName) { + throw new IOException(HASH_NAME_KEY + " must be provided as option"); + } + + try { + this.digest = MessageDigest.getInstance(hashName); + } catch (NoSuchAlgorithmException e) { + throw new IOException(e); + } + + this.topKey = null; + this.topValue = null; + this.source = source; + } + + @Override + public boolean hasTop() { + return null != topKey; + } + + @Override + public void next() throws IOException { + // We can't call next() if we already consumed it all + if (!this.source.hasTop()) { + this.topKey = null; + this.topValue = null; + return; + } + + this.source.next(); + + consume(); + } + + @Override + public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { + this.source.seek(range, columnFamilies, inclusive); + + consume(); + } + + protected void consume() throws IOException { + digest.reset(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + + if (!this.source.hasTop()) { + this.topKey = null; + this.topValue = null; + + return; + } + + Key lastKeySeen = null; + while (this.source.hasTop()) { + baos.reset(); + + Key currentKey = this.source.getTopKey(); + lastKeySeen = currentKey; + + currentKey.write(dos); + this.source.getTopValue().write(dos); + + digest.update(baos.toByteArray()); + + this.source.next(); + } + + this.topKey = lastKeySeen; + this.topValue = new Value(digest.digest()); + } + + @Override + public Key getTopKey() { + return topKey; + } + + @Override + public Value getTopValue() { + return topValue; + } + + @Override + public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { + DigestIterator copy = new DigestIterator(); + try { + copy.digest = MessageDigest.getInstance(digest.getAlgorithm()); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + + copy.topKey = this.topKey; + copy.topValue = this.topValue; + copy.source = this.source.deepCopy(env); + + return copy; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/scalability/Ingest.java b/core/src/main/java/org/apache/accumulo/testing/core/scalability/Ingest.java new file mode 100644 index 00000000..1a61eea1 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/scalability/Ingest.java @@ -0,0 +1,143 @@ +/* + * 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.testing.core.scalability; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.BatchWriterConfig; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.MutationsRejectedException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.security.ColumnVisibility; +import org.apache.accumulo.testing.core.continuous.ContinuousIngest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class Ingest extends ScaleTest { + + private static final Logger log = LoggerFactory.getLogger(Ingest.class); + + @Override + public void setup() { + + Connector conn = getConnector(); + String tableName = getTestProperty("TABLE"); + + // delete existing table + if (conn.tableOperations().exists(tableName)) { + System.out.println("Deleting existing table: " + tableName); + try { + conn.tableOperations().delete(tableName); + } catch (Exception e) { + log.error("Failed to delete table '" + tableName + "'.", e); + } + } + + // create table + try { + conn.tableOperations().create(tableName); + conn.tableOperations().addSplits(tableName, calculateSplits()); + conn.tableOperations().setProperty(tableName, "table.split.threshold", "256M"); + } catch (Exception e) { + log.error("Failed to create table '" + tableName + "'.", e); + } + + } + + @Override + public void client() { + + Connector conn = getConnector(); + String tableName = getTestProperty("TABLE"); + + // get batch writer configuration + long maxMemory = Long.parseLong(getTestProperty("MAX_MEMORY")); + long maxLatency = Long.parseLong(getTestProperty("MAX_LATENCY")); + int maxWriteThreads = Integer.parseInt(getTestProperty("NUM_THREADS")); + + // create batch writer + BatchWriter bw = null; + try { + bw = conn.createBatchWriter(tableName, new BatchWriterConfig().setMaxMemory(maxMemory).setMaxLatency(maxLatency, TimeUnit.MILLISECONDS) + .setMaxWriteThreads(maxWriteThreads)); + } catch (TableNotFoundException e) { + log.error("Table '" + tableName + "' not found.", e); + System.exit(-1); + } + + // configure writing + Random r = new Random(); + String ingestInstanceId = UUID.randomUUID().toString(); + long numIngestEntries = Long.parseLong(getTestProperty("NUM_ENTRIES")); + long minRow = 0L; + long maxRow = 9223372036854775807L; + int maxColF = 32767; + int maxColQ = 32767; + long count = 0; + long totalBytes = 0; + + ColumnVisibility cv = new ColumnVisibility(); + + // start timer + startTimer(); + + // write specified number of entries + while (count < numIngestEntries) { + count++; + long rowId = ContinuousIngest.genLong(minRow, maxRow, r); + Mutation m = ContinuousIngest.genMutation(rowId, r.nextInt(maxColF), r.nextInt(maxColQ), cv, ingestInstanceId.getBytes(UTF_8), count, null, false); + totalBytes += m.numBytes(); + try { + bw.addMutation(m); + } catch (MutationsRejectedException e) { + log.error("Mutations rejected.", e); + System.exit(-1); + } + } + + // close writer + try { + bw.close(); + } catch (MutationsRejectedException e) { + log.error("Could not close BatchWriter due to mutations being rejected.", e); + System.exit(-1); + } + + // stop timer + stopTimer(count, totalBytes); + } + + @Override + public void teardown() { + + Connector conn = getConnector(); + String tableName = getTestProperty("TABLE"); + + try { + conn.tableOperations().delete(tableName); + } catch (Exception e) { + log.error("Failed to delete table '" + tableName + "'", e); + } + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/scalability/Run.java b/core/src/main/java/org/apache/accumulo/testing/core/scalability/Run.java new file mode 100644 index 00000000..f7f74582 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/scalability/Run.java @@ -0,0 +1,97 @@ +/* + * 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.testing.core.scalability; + +import java.io.FileInputStream; +import java.net.InetAddress; +import java.util.Properties; + +import org.apache.accumulo.core.cli.Help; +import org.apache.accumulo.core.util.CachedConfiguration; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.beust.jcommander.Parameter; + +public class Run { + + private static final Logger log = LoggerFactory.getLogger(Run.class); + + static class Opts extends Help { + @Parameter(names = "--testId", required = true) + String testId; + @Parameter(names = "--action", required = true, description = "one of 'setup', 'teardown' or 'client'") + String action; + @Parameter(names = "--count", description = "number of tablet servers", required = true) + int numTabletServers; + } + + public static void main(String[] args) throws Exception { + + final String sitePath = "/tmp/scale-site.conf"; + final String testPath = "/tmp/scale-test.conf"; + Opts opts = new Opts(); + opts.parseArgs(Run.class.getName(), args); + + Configuration conf = CachedConfiguration.getInstance(); + FileSystem fs; + fs = FileSystem.get(conf); + + fs.copyToLocalFile(new Path("/accumulo-scale/conf/site.conf"), new Path(sitePath)); + fs.copyToLocalFile(new Path(String.format("/accumulo-scale/conf/%s.conf", opts.testId)), new Path(testPath)); + + // load configuration file properties + Properties scaleProps = new Properties(); + Properties testProps = new Properties(); + try { + FileInputStream fis = new FileInputStream(sitePath); + try { + scaleProps.load(fis); + } finally { + fis.close(); + } + fis = new FileInputStream(testPath); + try { + testProps.load(fis); + } finally { + fis.close(); + } + } catch (Exception e) { + log.error("Error loading config file.", e); + } + + ScaleTest test = (ScaleTest) Class.forName(String.format("org.apache.accumulo.test.scalability.%s", opts.testId)).newInstance(); + + test.init(scaleProps, testProps, opts.numTabletServers); + + if (opts.action.equalsIgnoreCase("setup")) { + test.setup(); + } else if (opts.action.equalsIgnoreCase("client")) { + InetAddress addr = InetAddress.getLocalHost(); + String host = addr.getHostName(); + fs.createNewFile(new Path("/accumulo-scale/clients/" + host)); + test.client(); + fs.copyFromLocalFile(new Path("/tmp/scale.out"), new Path("/accumulo-scale/results/" + host)); + } else if (opts.action.equalsIgnoreCase("teardown")) { + test.teardown(); + } + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/scalability/ScaleTest.java b/core/src/main/java/org/apache/accumulo/testing/core/scalability/ScaleTest.java new file mode 100644 index 00000000..a78f39ec --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/scalability/ScaleTest.java @@ -0,0 +1,88 @@ +/* + * 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.testing.core.scalability; + +import java.util.Properties; +import java.util.TreeSet; + +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.ClientConfiguration; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.ZooKeeperInstance; +import org.apache.accumulo.core.client.security.tokens.PasswordToken; +import org.apache.hadoop.io.Text; + +public abstract class ScaleTest { + + private Connector conn; + private Properties scaleProps; + private Properties testProps; + private int numTabletServers; + private long startTime; + + public void init(Properties scaleProps, Properties testProps, int numTabletServers) throws AccumuloException, AccumuloSecurityException { + + this.scaleProps = scaleProps; + this.testProps = testProps; + this.numTabletServers = numTabletServers; + + // get properties to create connector + String instanceName = this.scaleProps.getProperty("INSTANCE_NAME"); + String zookeepers = this.scaleProps.getProperty("ZOOKEEPERS"); + String user = this.scaleProps.getProperty("USER"); + String password = this.scaleProps.getProperty("PASSWORD"); + System.out.println(password); + + conn = new ZooKeeperInstance(new ClientConfiguration().withInstance(instanceName).withZkHosts(zookeepers)).getConnector(user, new PasswordToken(password)); + } + + protected void startTimer() { + startTime = System.currentTimeMillis(); + } + + protected void stopTimer(long numEntries, long numBytes) { + long endTime = System.currentTimeMillis(); + System.out.printf("ELAPSEDMS %d %d %d%n", endTime - startTime, numEntries, numBytes); + } + + public abstract void setup(); + + public abstract void client(); + + public abstract void teardown(); + + public TreeSet calculateSplits() { + int numSplits = numTabletServers - 1; + long distance = (Long.MAX_VALUE / numTabletServers) + 1; + long split = distance; + TreeSet keys = new TreeSet<>(); + for (int i = 0; i < numSplits; i++) { + keys.add(new Text(String.format("%016x", split))); + split += distance; + } + return keys; + } + + public Connector getConnector() { + return conn; + } + + public String getTestProperty(String key) { + return testProps.getProperty(key); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/DataWriter.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/DataWriter.java new file mode 100644 index 00000000..e7158e21 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/DataWriter.java @@ -0,0 +1,50 @@ +/* + * 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.stress.random; + +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.MutationsRejectedException; + +public class DataWriter extends Stream { + private final BatchWriter writer; + private final RandomMutations mutations; + + public DataWriter(BatchWriter writer, RandomMutations mutations) { + this.writer = writer; + this.mutations = mutations; + } + + @Override + public Void next() { + try { + writer.addMutation(mutations.next()); + } catch (MutationsRejectedException e) { + throw new RuntimeException(e); + } + return null; + } + + @Override + public void finalize() { + try { + this.writer.close(); + } catch (MutationsRejectedException e) { + System.err.println("Error closing batch writer."); + e.printStackTrace(); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/IntArgValidator.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/IntArgValidator.java new file mode 100644 index 00000000..5a5ad3e7 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/IntArgValidator.java @@ -0,0 +1,34 @@ +/* + * 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.stress.random; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +import com.beust.jcommander.IValueValidator; +import com.beust.jcommander.ParameterException; + +public class IntArgValidator implements IValueValidator { + + @Override + public void validate(String name, Integer value) throws ParameterException { + requireNonNull(value); + checkArgument(value > 0); + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomByteArrays.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomByteArrays.java new file mode 100644 index 00000000..405fabb2 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomByteArrays.java @@ -0,0 +1,33 @@ +/* + * 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.stress.random; + +/** + * A stream that will create random byte arrays as it is looped over. + */ +public class RandomByteArrays extends Stream { + private final RandomWithinRange random_arrays; + + public RandomByteArrays(RandomWithinRange random_arrays) { + this.random_arrays = random_arrays; + } + + @Override + public byte[] next() { + return random_arrays.next_bytes(); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomMutations.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomMutations.java new file mode 100644 index 00000000..db5da557 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomMutations.java @@ -0,0 +1,56 @@ +/* + * 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.stress.random; + +import org.apache.accumulo.core.data.Mutation; + +public class RandomMutations extends Stream { + private final RandomByteArrays rows, column_families, column_qualifiers, values; + private final RandomWithinRange row_widths; + private final int max_cells_per_mutation; + private byte[] current_row; + private int cells_remaining_in_row; + + public RandomMutations(RandomByteArrays rows, RandomByteArrays column_families, RandomByteArrays column_qualifiers, RandomByteArrays values, + RandomWithinRange row_widths, int max_cells_per_mutation) { + this.rows = rows; + this.column_families = column_families; + this.column_qualifiers = column_qualifiers; + this.values = values; + this.row_widths = row_widths; + this.max_cells_per_mutation = (max_cells_per_mutation > 0 ? max_cells_per_mutation : Integer.MAX_VALUE); + + current_row = null; + cells_remaining_in_row = 0; + } + + // TODO should we care about timestamps? + @Override + public Mutation next() { + if (cells_remaining_in_row == 0) { + current_row = rows.next(); + cells_remaining_in_row = row_widths.next(); + } + Mutation m = new Mutation(current_row); + final int cells = Math.min(cells_remaining_in_row, max_cells_per_mutation); + for (int i = 1; i <= cells; i++) { + m.put(column_families.next(), column_qualifiers.next(), values.next()); + } + cells_remaining_in_row -= cells; + return m; + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomWithinRange.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomWithinRange.java new file mode 100644 index 00000000..06cea283 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/RandomWithinRange.java @@ -0,0 +1,58 @@ +/* + * 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.stress.random; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.Random; + +/** + * Class that returns positive integers between some minimum and maximum. + * + */ +public class RandomWithinRange { + private final Random random; + private final int min, max; + + public RandomWithinRange(int seed, int min, int max) { + this(new Random(seed), min, max); + } + + public RandomWithinRange(Random random, int min, int max) { + checkArgument(min > 0, "Min must be positive."); + checkArgument(max >= min, "Max must be greater than or equal to min."); + this.random = random; + this.min = min; + this.max = max; + } + + public int next() { + if (min == max) { + return min; + } else { + // we pick a random number that's between 0 and (max - min), then add + // min as an offset to get a random number that's [min, max) + return random.nextInt(max - min) + min; + } + } + + public byte[] next_bytes() { + byte[] b = new byte[next()]; + random.nextBytes(b); + return b; + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/Scan.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/Scan.java new file mode 100644 index 00000000..3e8d5fd4 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/Scan.java @@ -0,0 +1,121 @@ +/* + * 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.stress.random; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Random; + +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.hadoop.io.Text; + +import com.google.common.collect.Lists; + +public class Scan { + + public static void main(String[] args) throws Exception { + ScanOpts opts = new ScanOpts(); + opts.parseArgs(Scan.class.getName(), args); + + Connector connector = opts.getConnector(); + Scanner scanner = connector.createScanner(opts.getTableName(), new Authorizations()); + + if (opts.isolate) { + scanner.enableIsolation(); + } + + Random tablet_index_generator = new Random(opts.scan_seed); + + LoopControl scanning_condition = opts.continuous ? new ContinuousLoopControl() : new IterativeLoopControl(opts.scan_iterations); + + while (scanning_condition.keepScanning()) { + Range range = pickRange(connector.tableOperations(), opts.getTableName(), tablet_index_generator); + scanner.setRange(range); + if (opts.batch_size > 0) { + scanner.setBatchSize(opts.batch_size); + } + try { + consume(scanner); + } catch (Exception e) { + System.err.println(String.format("Exception while scanning range %s. Check the state of Accumulo for errors.", range)); + throw e; + } + } + } + + public static void consume(Iterable iterable) { + Iterator itr = iterable.iterator(); + while (itr.hasNext()) { + itr.next(); + } + } + + public static Range pickRange(TableOperations tops, String table, Random r) throws TableNotFoundException, AccumuloSecurityException, AccumuloException { + ArrayList splits = Lists.newArrayList(tops.listSplits(table)); + if (splits.isEmpty()) { + return new Range(); + } else { + int index = r.nextInt(splits.size()); + Text endRow = splits.get(index); + Text startRow = index == 0 ? null : splits.get(index - 1); + return new Range(startRow, false, endRow, true); + } + } + + /* + * These interfaces + implementations are used to determine how many times the scanner should look up a random tablet and scan it. + */ + static interface LoopControl { + public boolean keepScanning(); + } + + // Does a finite number of iterations + static class IterativeLoopControl implements LoopControl { + private final int max; + private int current; + + public IterativeLoopControl(int max) { + this.max = max; + this.current = 0; + } + + @Override + public boolean keepScanning() { + if (current < max) { + ++current; + return true; + } else { + return false; + } + } + } + + // Does an infinite number of iterations + static class ContinuousLoopControl implements LoopControl { + @Override + public boolean keepScanning() { + return true; + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/ScanOpts.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/ScanOpts.java new file mode 100644 index 00000000..e3f73f76 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/ScanOpts.java @@ -0,0 +1,46 @@ +/* + * 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.stress.random; + +import org.apache.accumulo.core.cli.ClientOnDefaultTable; + +import com.beust.jcommander.Parameter; + +class ScanOpts extends ClientOnDefaultTable { + @Parameter(names = "--isolate", description = "true to turn on scan isolation, false to turn off. default is false.") + boolean isolate = false; + + @Parameter(names = "--num-iterations", description = "number of scan iterations") + int scan_iterations = 1024; + + @Parameter(names = "--continuous", description = "continuously scan the table. note that this overrides --num-iterations") + boolean continuous; + + @Parameter(names = "--scan-seed", description = "seed for randomly choosing tablets to scan") + int scan_seed = 1337; + + @Parameter(names = "--scan-batch-size", description = "scanner batch size") + int batch_size = -1; + + public ScanOpts() { + this(WriteOptions.DEFAULT_TABLE); + } + + public ScanOpts(String table) { + super(table); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/Stream.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/Stream.java new file mode 100644 index 00000000..72b31e51 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/Stream.java @@ -0,0 +1,40 @@ +/* + * 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.stress.random; + +import java.util.Iterator; + +/** + * Base class to model an infinite stream of data. A stream implements an iterator whose {{@link #hasNext()} method will always return true. + * + */ +public abstract class Stream implements Iterator { + + @Override + public final boolean hasNext() { + return true; + } + + @Override + public abstract T next(); + + @Override + public final void remove() { + throw new UnsupportedOperationException(); + } + +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/Write.java new file mode 100644 index 00000000..ea6f1646 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/Write.java @@ -0,0 +1,77 @@ +/* + * 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.stress.random; + +import org.apache.accumulo.core.cli.BatchWriterOpts; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.TableExistsException; +import org.apache.accumulo.core.client.TableNotFoundException; + +public class Write { + + public static void main(String[] args) throws Exception { + WriteOptions opts = new WriteOptions(); + BatchWriterOpts batch_writer_opts = new BatchWriterOpts(); + opts.parseArgs(Write.class.getName(), args, batch_writer_opts); + + opts.check(); + + Connector c = opts.getConnector(); + + if (opts.clear_table && c.tableOperations().exists(opts.getTableName())) { + try { + c.tableOperations().delete(opts.getTableName()); + } catch (TableNotFoundException e) { + System.err.println("Couldn't delete the table because it doesn't exist any more."); + } + } + + if (!c.tableOperations().exists(opts.getTableName())) { + try { + c.tableOperations().create(opts.getTableName()); + } catch (TableExistsException e) { + System.err.println("Couldn't create table ourselves, but that's ok. Continuing."); + } + } + + long writeDelay = opts.write_delay; + if (writeDelay < 0) { + writeDelay = 0; + } + + DataWriter dw = new DataWriter(c.createBatchWriter(opts.getTableName(), batch_writer_opts.getBatchWriterConfig()), new RandomMutations( + // rows + new RandomByteArrays(new RandomWithinRange(opts.row_seed, opts.rowMin(), opts.rowMax())), + // cfs + new RandomByteArrays(new RandomWithinRange(opts.cf_seed, opts.cfMin(), opts.cfMax())), + // cqs + new RandomByteArrays(new RandomWithinRange(opts.cq_seed, opts.cqMin(), opts.cqMax())), + // vals + new RandomByteArrays(new RandomWithinRange(opts.value_seed, opts.valueMin(), opts.valueMax())), + // number of cells per row + new RandomWithinRange(opts.row_width_seed, opts.rowWidthMin(), opts.rowWidthMax()), + // max cells per mutation + opts.max_cells_per_mutation)); + + while (true) { + dw.next(); + if (writeDelay > 0) { + Thread.sleep(writeDelay); + } + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/WriteOptions.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/WriteOptions.java new file mode 100644 index 00000000..f92a9ebe --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/WriteOptions.java @@ -0,0 +1,169 @@ +/* + * 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.stress.random; + +import org.apache.accumulo.core.cli.ClientOnDefaultTable; + +import com.beust.jcommander.Parameter; + +class WriteOptions extends ClientOnDefaultTable { + static final String DEFAULT_TABLE = "stress_test"; + static final int DEFAULT_MIN = 1, DEFAULT_MAX = 128, DEFAULT_SPREAD = DEFAULT_MAX - DEFAULT_MIN; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--min-row-size", description = "minimum row size") + Integer row_min; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--max-row-size", description = "maximum row size") + Integer row_max; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--min-cf-size", description = "minimum column family size") + Integer cf_min; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--max-cf-size", description = "maximum column family size") + Integer cf_max; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--min-cq-size", description = "minimum column qualifier size") + Integer cq_min; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--max-cq-size", description = "maximum column qualifier size") + Integer cq_max; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--min-value-size", description = "minimum value size") + Integer value_min; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--max-value-size", description = "maximum value size") + Integer value_max; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--min-row-width", description = "minimum row width") + Integer row_width_min; + + @Parameter(validateValueWith = IntArgValidator.class, names = "--max-row-width", description = "maximum row width") + Integer row_width_max; + + @Parameter(names = "--clear-table", description = "clears the table before ingesting") + boolean clear_table; + + @Parameter(names = "--row-seed", description = "seed for generating rows") + int row_seed = 87; + + @Parameter(names = "--cf-seed", description = "seed for generating column families") + int cf_seed = 7; + + @Parameter(names = "--cq-seed", description = "seed for generating column qualifiers") + int cq_seed = 43; + + @Parameter(names = "--value-seed", description = "seed for generating values") + int value_seed = 99; + + @Parameter(names = "--row-width-seed", description = "seed for generating the number of cells within a row (a row's \"width\")") + int row_width_seed = 444; + + @Parameter(names = "--max-cells-per-mutation", description = "maximum number of cells per mutation; non-positive value implies no limit") + int max_cells_per_mutation = -1; + + @Parameter(names = "--write-delay", description = "milliseconds to wait between writes") + long write_delay = 0L; + + public WriteOptions(String table) { + super(table); + } + + public WriteOptions() { + this(DEFAULT_TABLE); + } + + private static int minOrDefault(Integer ref) { + return ref == null ? DEFAULT_MIN : ref; + } + + private static int calculateMax(Integer min_ref, Integer max_ref) { + if (max_ref == null) { + if (min_ref == null) { + return DEFAULT_MAX; + } else { + return min_ref + DEFAULT_SPREAD; + } + } else { + return max_ref; + } + } + + public void check() { + checkPair("ROW", row_min, row_max); + checkPair("COLUMN FAMILY", cf_min, cf_max); + checkPair("COLUMN QUALIFIER", cq_min, cq_max); + checkPair("VALUE", value_min, value_max); + } + + public void checkPair(String label, Integer min_ref, Integer max_ref) { + // we've already asserted that the numbers will either be + // 1) null + // 2) positive + // need to verify that they're coherent here + + if (min_ref == null && max_ref != null) { + // we don't support just specifying a max yet + throw new IllegalArgumentException(String.format("[%s] Maximum value supplied, but no minimum. Must supply a minimum with a maximum value.", label)); + } else if (min_ref != null && max_ref != null) { + // if a user supplied lower and upper bounds, we need to verify + // that min <= max + if (min_ref.compareTo(max_ref) > 0) { + throw new IllegalArgumentException(String.format("[%s] Min value (%d) is greater than max value (%d)", label, min_ref, max_ref)); + } + } + } + + public int rowMin() { + return minOrDefault(row_min); + } + + public int rowMax() { + return calculateMax(row_min, row_max); + } + + public int cfMin() { + return minOrDefault(cf_min); + } + + public int cfMax() { + return calculateMax(cf_min, cf_max); + } + + public int cqMin() { + return minOrDefault(cq_min); + } + + public int cqMax() { + return calculateMax(cq_min, cq_max); + } + + public int valueMin() { + return minOrDefault(value_min); + } + + public int valueMax() { + return calculateMax(value_min, value_max); + } + + public int rowWidthMin() { + return minOrDefault(row_width_min); + } + + public int rowWidthMax() { + return calculateMax(row_width_min, row_width_max); + } +} diff --git a/core/src/main/java/org/apache/accumulo/testing/core/stress/package-info.java b/core/src/main/java/org/apache/accumulo/testing/core/stress/package-info.java new file mode 100644 index 00000000..fdbf72ef --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/testing/core/stress/package-info.java @@ -0,0 +1,36 @@ +/* + * 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. + */ +/** + * This package contains utility classes designed to test Accumulo when large cells are being written. This is an attempt to observe the behavior Accumulo + * displays when compacting and reading these cells. + * + * There are two components to this package: {@link org.apache.accumulo.test.stress.random.Write} and {@link org.apache.accumulo.test.stress.random.Scan}. + * + * The {@link org.apache.accumulo.test.stress.random.Write} provides facilities for writing random sized cells. Users can configure minimum and maximum + * sized portions of a cell. The portions users can configure are the row, column family, column qualifier and value. Note that the sizes are uniformly + * distributed between the minimum and maximum values. See {@link org.apache.accumulo.test.stress.random.WriteOptions} for available options and default sizing + * information. + * + * The Scan provides users with the ability to query tables generated by the Write. It will pick a tablet at random and scan the entire range. The + * amount of times this process is done is user configurable. By default, it happens 1,024 times. Users can also specify whether or not the scan should be + * isolated or not. + * + * There is no shared state intended by either of these services. This allows multiple clients to be run in parallel, either on the same host or distributed + * across hosts. + */ +package org.apache.accumulo.test.stress.random; + diff --git a/test/agitator/.gitignore b/test/agitator/.gitignore new file mode 100644 index 00000000..3429b013 --- /dev/null +++ b/test/agitator/.gitignore @@ -0,0 +1,3 @@ +*~ +*.ini +*.pyc diff --git a/test/agitator/README.md b/test/agitator/README.md new file mode 100644 index 00000000..8abb74c5 --- /dev/null +++ b/test/agitator/README.md @@ -0,0 +1,39 @@ + + +Agitator: randomly kill processes +=========================== + +The agitator is used to randomly select processes for termination during +system test. + +Configure the agitator using the example agitator.ini file provided. + +Create a list of hosts to be agitated: + + $ cp ../../../conf/tservers hosts + $ echo master >> hosts + $ echo namenode >> hosts + +The agitator can be used to kill and restart any part of the accumulo +ecosystem: zookeepers, namenode, datanodes, tablet servers and master. +You can choose to agitate them all with "--all" + + $ ./agitator.py --all --hosts=hosts --config=agitator.ini --log DEBUG + +You will need to be able to ssh, without passwords, to all your hosts as +the user that can kill and start the services. diff --git a/test/agitator/agitator.ini.example b/test/agitator/agitator.ini.example new file mode 100644 index 00000000..35125613 --- /dev/null +++ b/test/agitator/agitator.ini.example @@ -0,0 +1,56 @@ +# 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. + +[DEFAULT] +install=%(env.pwd)s/../../../.. +user=%(env.user)s + +[agitator] +kill=kill -9 +ssh=ssh -q -A -o StrictHostKeyChecking=no +sleep=300 +sleep.restart=30 +sleep.jitter=30 + +[accumulo] +home=%(install)s/accumulo +tserver.kill.min=1 +tserver.kill.max=1 +tserver.frequency=0.8 + +master.kill.min=1 +master.kill.max=1 +master.frequency=0.1 + +gc.kill.min=1 +gc.kill.max=1 +gc.frequency=0.1 + +[hadoop] +home=%(install)s/hadoop +bin=%(home)s/bin +datanode.frequency=0.8 +datanode.kill.min=1 +datanode.kill.max=1 +namenode.frequency=0.05 +namenode.kill.min=1 +namenode.kill.max=1 +secondarynamenode.frequency=0.05 +secondarynamenode.kill.min=1 +secondarynamenode.kill.max=1 + +[zookeeper] +home=%(install)s/zookeeper +frequency=0.05 diff --git a/test/agitator/agitator.py b/test/agitator/agitator.py new file mode 100755 index 00000000..db945464 --- /dev/null +++ b/test/agitator/agitator.py @@ -0,0 +1,241 @@ +#! /usr/bin/python + +# 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. + +import random +import logging +import ConfigParser + +# add the environment variables as default settings +import os +defaults=dict([('env.' + k, v) for k, v in os.environ.iteritems()]) +config = ConfigParser.ConfigParser(defaults) + +# things you can do to a particular kind of process +class Proc: + program = 'Unknown' + _frequencyToKill = 1.0 + + def start(self, host): + pass + + def find(self, host): + pass + + def numberToKill(self): + return (1, 1) + + def frequencyToKill(self): + return self._frequencyToKill + + def user(self): + return config.get(self.program, 'user') + + def kill(self, host, pid): + kill = config.get('agitator', 'kill').split() + code, stdout, stderr = self.runOn(host, kill + [pid]) + if code != 0: + raise logging.warn("Unable to kill %d on %s (%s)", pid, host, stderr) + + def runOn(self, host, cmd): + ssh = config.get('agitator', 'ssh').split() + return self.run(ssh + ["%s@%s" % (self.user(), host)] + cmd) + + def run(self, cmd): + import subprocess + cmd = map(str, cmd) + logging.debug('Running %s', cmd) + p = subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + stdout, stderr = p.communicate() + if stdout.strip(): + logging.debug("%s", stdout.strip()) + if stderr.strip(): + logging.error("%s", stderr.strip()) + if p.returncode != 0: + logging.error("Problem running %s", ' '.join(cmd)) + return p.returncode, stdout, stderr + + def __repr__(self): + return self.program + +class Zookeeper(Proc): + program = 'zookeeper' + def __init__(self): + self._frequencyToKill = config.getfloat(self.program, 'frequency') + + def start(self, host): + self.runOn(host, [config.get(self.program, 'home') + '/bin/zkServer.sh start']) + + def find(self, host): + code, stdout, stderr = self.runOn(host, ['pgrep -f [Q]uorumPeerMain || true']) + return map(int, [line for line in stdout.split("\n") if line]) + +class Hadoop(Proc): + section = 'hadoop' + def __init__(self, program): + self.program = program + self._frequencyToKill = config.getfloat(self.section, program + '.frequency') + self.minimumToKill = config.getint(self.section, program + '.kill.min') + self.maximumToKill = config.getint(self.section, program + '.kill.max') + + def start(self, host): + binDir = config.get(self.section, 'bin') + self.runOn(host, ['nohup %s/hdfs %s < /dev/null >/dev/null 2>&1 &' %(binDir, self.program)]) + + def find(self, host): + code, stdout, stderr = self.runOn(host, ["pgrep -f 'proc[_]%s' || true" % (self.program,)]) + return map(int, [line for line in stdout.split("\n") if line]) + + def numberToKill(self): + return (self.minimumToKill, self.maximumToKill) + + def user(self): + return config.get(self.section, 'user') + +class Accumulo(Hadoop): + section = 'accumulo' + def start(self, host): + home = config.get(self.section, 'home') + self.runOn(host, ['nohup %s/bin/accumulo %s /dev/null 2>&1 & ' %(home, self.program)]) + + def find(self, host): + code, stdout, stderr = self.runOn(host, ["pgrep -f 'app[=]%s' || true" % self.program]) + return map(int, [line for line in stdout.split("\n") if line]) + +def fail(msg): + import sys + logging.critical(msg) + sys.exit(1) + +def jitter(n): + return random.random() * n - n / 2 + +def sleep(n): + if n > 0: + logging.info("Sleeping %.2f", n) + import time + time.sleep(n) + +def agitate(hosts, procs): + starters = [] + + logging.info("Agitating %s on %d hosts" % (procs, len(hosts))) + + section = 'agitator' + + # repeatedly... + while True: + if starters: + # start up services that were previously killed + t = max(0, config.getfloat(section, 'sleep.restart') + jitter(config.getfloat(section, 'sleep.jitter'))) + sleep(t) + for host, proc in starters: + logging.info('Starting %s on %s', proc, host) + proc.start(host) + starters = [] + + # wait some time + t = max(0, config.getfloat(section, 'sleep') + jitter(config.getfloat(section, 'sleep.jitter'))) + sleep(t) + + # for some processes + for p in procs: + + # roll dice: should it be killed? + if random.random() < p.frequencyToKill(): + + # find them + from multiprocessing import Pool + def finder(host): + return host, p.find(host) + with Pool(5) as pool: + result = pool.map(finder, hosts) + candidates = {} + for host, pids in result: + if pids: + candidates[host] = pids + + # how many? + minKill, maxKill = p.numberToKill() + count = min(random.randrange(minKill, maxKill + 1), len(candidates)) + + # pick the victims + doomedHosts = random.sample(candidates.keys(), count) + + # kill them + logging.info("Killing %s on %s", p, doomedHosts) + for doomedHost in doomedHosts: + pids = candidates[doomedHost] + if not pids: + logging.error("Unable to kill any %s on %s: no processes of that type are running", p, doomedHost) + else: + pid = random.choice(pids) + logging.debug("Killing %s (%d) on %s", p, pid, doomedHost) + p.kill(doomedHost, pid) + # remember to restart them later + starters.append((doomedHost, p)) + +def main(): + import argparse + parser = argparse.ArgumentParser(description='Kill random processes') + parser.add_argument('--log', help='set the log level', default='INFO') + parser.add_argument('--namenodes', help='randomly kill namenodes', action="store_true") + parser.add_argument('--secondary', help='randomly kill secondary namenode', action="store_true") + parser.add_argument('--datanodes', help='randomly kill datanodes', action="store_true") + parser.add_argument('--tservers', help='randomly kill tservers', action="store_true") + parser.add_argument('--masters', help='randomly kill masters', action="store_true") + parser.add_argument('--zookeepers', help='randomly kill zookeepers', action="store_true") + parser.add_argument('--gc', help='randomly kill the file garbage collector', action="store_true") + parser.add_argument('--all', + help='kill any of the tservers, masters, datanodes, namenodes or zookeepers', + action='store_true') + parser.add_argument('--hosts', type=argparse.FileType('r'), required=True) + parser.add_argument('--config', type=argparse.FileType('r'), required=True) + args = parser.parse_args() + + config.readfp(args.config) + + level = getattr(logging, args.log.upper(), None) + if isinstance(level, int): + logging.basicConfig(level=level) + + procs = [] + def addIf(flag, proc): + if flag or args.all: + procs.append(proc) + + addIf(args.namenodes, Hadoop('namenode')) + addIf(args.datanodes, Hadoop('datanode')) + addIf(args.secondary, Hadoop('secondarynamenode')) + addIf(args.tservers, Accumulo('tserver')) + addIf(args.masters, Accumulo('master')) + addIf(args.gc, Accumulo('gc')) + addIf(args.zookeepers, Zookeeper()) + if len(procs) == 0: + fail("No processes to agitate!\n") + + hosts = [] + for line in args.hosts.readlines(): + line = line.strip() + if line and line[0] != '#': + hosts.append(line) + if not hosts: + fail('No hosts to agitate!\n') + + agitate(hosts, procs) + +if __name__ == '__main__': + main() diff --git a/test/agitator/hosts.example b/test/agitator/hosts.example new file mode 100644 index 00000000..63fb8bb8 --- /dev/null +++ b/test/agitator/hosts.example @@ -0,0 +1,16 @@ +# 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. + +localhost diff --git a/test/bench/README.md b/test/bench/README.md new file mode 100644 index 00000000..0929bc3f --- /dev/null +++ b/test/bench/README.md @@ -0,0 +1,61 @@ + +Benchmark Tests +=============== + +Running the Benchmarks +---------------------- + +Syntax for running run.py: + +> `$ ./run.py [-l -v -s -u -p -i ] [Benchmark1 ... BenchmarkN]` + +Specifying a specific benchmark or set of benchmarks runs only those, while +not specifying any runs all benchmarks. + +`-l` lists the benchmarks that will be run +`-v ` can either be slow, medium or fast +`-s ` is a number representing the verbosity of the debugging output: 10 is debug, 20 is info, 30 is warning, etc. +`-u ` user to use when connecting with accumulo. If not set you will be prompted to input it. +`-p ` password to use when connecting with accumulo. If not set you will be prompted to input it. +`-z ` comma delimited lit of zookeeper host:port pairs to use when connecting with accumulo. If not set you will be prompted to input it. +`-i ` instance to use when connecting with accumulo. If not set you will be prompted to input it. + +The Benchmarks +-------------- + +Values in a 3-tuple are the slow,medium,fast speeds at which you can run the benchmarks. + +* CloudStone1: Test the speed at which we can check that accumulo is up and we can reach all the tservers. Lower is better. +* CloudStone2: Ingest 10000,100000,1000000 rows of values 50 bytes on every tserver. Higher is better. +* CloudStone3: Ingest 1000,5000,10000 rows of values 1024,8192,65535 bytes on every tserver. Higher is better. +* CloudStone4 (TeraSort): Ingests 10000,10000000,10000000000 rows. Lower score is better. +* CloudStone5: Creates 100,500,1000 tables named TestTableX and then deletes them. Lower is better. +* CloudStone6: Creates a table with 400, 800, 1000 splits. Lower is better. + +Terasort +-------- + +The 4th Benchmark is Terasort. Run the benchmarks with speed 'slow' to do a full terasort. + +Misc +---- + +These benchmarks create tables in accumulo named 'test_ingest' and 'CloudIngestTest'. These tables are deleted +at the end of the benchmarks. The benchmarks will also alter user auths while it runs. It is recommended that +a benchmark user is created. + diff --git a/test/bench/cloudstone1/__init__.py b/test/bench/cloudstone1/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone1/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone1/cloudstone1.py b/test/bench/cloudstone1/cloudstone1.py new file mode 100755 index 00000000..309ef9c5 --- /dev/null +++ b/test/bench/cloudstone1/cloudstone1.py @@ -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. + +import unittest + +import time + +from lib import cloudshell +from lib.Benchmark import Benchmark +from lib.tservers import runAll +from lib.path import accumulo + +class CloudStone1(Benchmark): + + def shortDescription(self): + return 'Test the speed at which we can check that accumulo is up '\ + 'and we can reach all the tservers. Lower is better.' + + def runTest(self): + code, out, err = cloudshell.run(self.username, self.password, 'table accumulo.metadata\nscan\n') + self.assertEqual(code, 0, "Could not scan the metadata table. %s %s" % (out, err)) + results = runAll('echo help | %s shell -u %s -p %s' % + (accumulo('bin', 'accumulo'), self.username, self.password)) + + def setSpeed(self, speed): + "We want to override this method but no speed can be set" + +def suite(): + result = unittest.TestSuite([ + CloudStone1(), + ]) + return result diff --git a/test/bench/cloudstone2/__init__.py b/test/bench/cloudstone2/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone2/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone2/cloudstone2.py b/test/bench/cloudstone2/cloudstone2.py new file mode 100755 index 00000000..996e5eff --- /dev/null +++ b/test/bench/cloudstone2/cloudstone2.py @@ -0,0 +1,49 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.IngestBenchmark import IngestBenchmark + +class CloudStone2(IngestBenchmark): + "TestIngest one million small records on each tserver" + + _size = 50 + _count = 1000000 + + def size(self): + return self._size + + def count(self): + return self._count + + def setSpeed(self, speed): + if speed == "fast": + self._size = 50 + self._count = 10000 + elif speed == "medium": + self._size = 50 + self._count = 100000 + elif speed == "slow": + self._size = 50 + self._count = 1000000 + +def suite(): + result = unittest.TestSuite([ + CloudStone2(), + ]) + return result diff --git a/test/bench/cloudstone3/__init__.py b/test/bench/cloudstone3/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone3/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone3/cloudstone3.py b/test/bench/cloudstone3/cloudstone3.py new file mode 100755 index 00000000..e6e1bca3 --- /dev/null +++ b/test/bench/cloudstone3/cloudstone3.py @@ -0,0 +1,50 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.IngestBenchmark import IngestBenchmark + +class CloudStone3(IngestBenchmark): + "TestIngest one thousand chunky records on each tserver" + + _size = 65535 + _count = 10000 + + def size(self): + return self._size + + def count(self): + return self._count + + def setSpeed(self, speed): + if speed == "fast": + self._size = 2**10 + self._count = 1000 + elif speed == "medium": + self._size = 2**13 + self._count = 5000 + elif speed == "slow": + self._size = 2**16 + self._count = 10000 + + +def suite(): + result = unittest.TestSuite([ + CloudStone3(), + ]) + return result diff --git a/test/bench/cloudstone4/__init__.py b/test/bench/cloudstone4/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone4/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone4/cloudstone4.py b/test/bench/cloudstone4/cloudstone4.py new file mode 100755 index 00000000..c87bec92 --- /dev/null +++ b/test/bench/cloudstone4/cloudstone4.py @@ -0,0 +1,29 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.TeraSortBenchmark import TeraSortBenchmark + +class CloudStone4(TeraSortBenchmark): + "TestCloudIngest one terabyte of data" + +def suite(): + result = unittest.TestSuite([ + CloudStone4(), + ]) + return result diff --git a/test/bench/cloudstone5/__init__.py b/test/bench/cloudstone5/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone5/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone5/cloudstone5.py b/test/bench/cloudstone5/cloudstone5.py new file mode 100755 index 00000000..2c4ba783 --- /dev/null +++ b/test/bench/cloudstone5/cloudstone5.py @@ -0,0 +1,29 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.TableSplitsBenchmark import TableSplitsBenchmark + +class CloudStone5(TableSplitsBenchmark): + "Creates a table with many splits" + +def suite(): + result = unittest.TestSuite([ + CloudStone5(), + ]) + return result diff --git a/test/bench/cloudstone6/__init__.py b/test/bench/cloudstone6/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone6/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone6/cloudstone6.py b/test/bench/cloudstone6/cloudstone6.py new file mode 100755 index 00000000..6e72633f --- /dev/null +++ b/test/bench/cloudstone6/cloudstone6.py @@ -0,0 +1,29 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.CreateTablesBenchmark import CreateTablesBenchmark + +class CloudStone6(CreateTablesBenchmark): + "Creates many tables and then deletes them" + +def suite(): + result = unittest.TestSuite([ + CloudStone6(), + ]) + return result diff --git a/test/bench/cloudstone7/__init__.py b/test/bench/cloudstone7/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone7/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone7/cloudstone7.py b/test/bench/cloudstone7/cloudstone7.py new file mode 100755 index 00000000..1933a4b1 --- /dev/null +++ b/test/bench/cloudstone7/cloudstone7.py @@ -0,0 +1,29 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.RowHashBenchmark import RowHashBenchmark + +class CloudStone7(RowHashBenchmark): + "Hashes all the rows in an accumulo table and outputs them to another table" + +def suite(): + result = unittest.TestSuite([ + CloudStone7(), + ]) + return result diff --git a/test/bench/cloudstone8/__init__.py b/test/bench/cloudstone8/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/cloudstone8/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/cloudstone8/cloudstone8.py b/test/bench/cloudstone8/cloudstone8.py new file mode 100755 index 00000000..a02a0a1c --- /dev/null +++ b/test/bench/cloudstone8/cloudstone8.py @@ -0,0 +1,64 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.TeraSortBenchmark import TeraSortBenchmark + +class CloudStone8(TeraSortBenchmark): + "Tests variable length input keys and values" + + keymin = 10 + keymax = 50 + valmin = 100 + valmax = 500 + rows = 1000000 + tablename = 'VariableLengthIngestTable' + + + def shortDescription(self): + return 'Ingests %d rows of variable key and value length to be sorted. '\ + 'Lower score is better.' % (self.numrows()) + + def setSpeed(self, speed): + if speed == "slow": + self.rows = 1000000 + self.keymin = 60 + self.keymax = 100 + self.valmin = 200 + self.valmax = 300 + self.numsplits = 400 + elif speed == "medium": + self.rows = 100000 + self.keymin = 40 + self.keymax = 70 + self.valmin = 130 + self.valmax = 170 + self.numsplits = 40 + elif speed == "fast": + self.rows = 10000 + self.keymin = 30 + self.keymax = 50 + self.valmin = 80 + self.valmax = 100 + self.numsplits = 4 + +def suite(): + result = unittest.TestSuite([ + CloudStone8(), + ]) + return result diff --git a/test/bench/lib/Benchmark.py b/test/bench/lib/Benchmark.py new file mode 100755 index 00000000..1481ccf4 --- /dev/null +++ b/test/bench/lib/Benchmark.py @@ -0,0 +1,115 @@ +# 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. + +import time + +import unittest +import os +import glob +import sys + +from options import log + +from path import accumulo + +class Benchmark(unittest.TestCase): + + username = '' + password = '' + zookeepers = '' + instance = '' + + def __init__(self): + unittest.TestCase.__init__(self) + self.finished = None + + def name(self): + return self.__class__.__name__ + + def setUp(self): + # verify accumulo is running + self.start = time.time() + + def tearDown(self): + self.stop = time.time() + log.debug("Runtime: %.2f", self.stop - self.start) + self.finished = True + + def runTime(self): + return self.stop - self.start + + def score(self): + if self.finished: + return self.runTime() + return 0. + + # Each class that extends Benchmark should overwrite this + def setSpeed(self, speed): + print "Classes that extend Benchmark need to override setSpeed." + + + def setUsername(self, user): + self.username = user + + def getUsername(self): + return self.username + + def setPassword(self, password): + self.password = password + + def getPassword(self): + return self.password + + def setZookeepers(self, zookeepers): + self.zookeepers = zookeepers + + def getZookeepers(self): + return self.zookeepers + + def setInstance(self, instance): + self.instance = instance + + def getInstance(self): + return self.instance + + def sleep(self, tts): + time.sleep(tts) + + def needsAuthentication(self): + return 0 + + def findjar(self, path): + globjar = [ j for j in glob.glob(path) if j.find('javadoc') == -1 and j.find('sources') == -1 ] + return globjar[0] + + # Returns the location of the local test jar + def gettestjar(self): + return self.findjar(accumulo() + '/lib/accumulo-test.jar') + + # Returns a string of core, thrift and zookeeper jars with a specified delim + def getjars(self, delim=','): + accumulo_core_jar = self.findjar(accumulo('lib', 'accumulo-core.jar')) + accumulo_start_jar = self.findjar(accumulo('lib', 'accumulo-start.jar')) + accumulo_fate_jar = self.findjar(accumulo('lib', 'accumulo-fate.jar')) + accumulo_trace_jar = self.findjar(accumulo('lib', 'accumulo-trace.jar')) + accumulo_thrift_jar = self.findjar(accumulo('lib', 'libthrift.jar')) + accumulo_zookeeper_jar = self.findjar(os.path.join(os.getenv('ZOOKEEPER_HOME'), 'zookeeper*.jar')) + return delim.join([accumulo_core_jar, accumulo_thrift_jar, accumulo_zookeeper_jar, accumulo_start_jar, + accumulo_fate_jar, accumulo_trace_jar]) + + # Builds the running command for the map/reduce class specified sans the arguments + def buildcommand(self, classname, *args): + return [accumulo('bin', 'accumulo'), classname, '-libjars', self.getjars()] + list(map(str, args)) + diff --git a/test/bench/lib/CreateTablesBenchmark.py b/test/bench/lib/CreateTablesBenchmark.py new file mode 100755 index 00000000..e5761d6d --- /dev/null +++ b/test/bench/lib/CreateTablesBenchmark.py @@ -0,0 +1,78 @@ +# 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. + +import unittest + +import subprocess + +from lib import cloudshell, runner, path +from lib.Benchmark import Benchmark +from lib.tservers import runEach, tserverNames +from lib.path import accumulo, accumuloJar +from lib.util import sleep +from lib.options import log + +class CreateTablesBenchmark(Benchmark): + "Creating and deleting tables" + + tables = 1000 + + def setUp(self): + for x in range(1, self.tables): + currentTable = 'test_ingest%d' % (x) + log.debug("Checking for table existence: %s" % currentTable) + code, out, err = cloudshell.run(self.username, self.password, 'table %s\n' % currentTable) + if out.find('does not exist') == -1: + command = 'deletetable -f %s\n' % (currentTable) + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, 'Did not successfully delete table: %s' % currentTable) + Benchmark.setUp(self) + + def runTest(self): + for x in range(1, self.tables): + currentTable = 'test_ingest%d' % (x) + command = 'createtable %s\n' % (currentTable) + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, 'Did not successfully create table: %s' % currentTable) + # print err + for x in range(1, self.tables): + currentTable = 'test_ingest%d' % (x) + command = 'deletetable -f %s\n' % (currentTable) + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, 'Did not successfully delete table: %s' % currentTable) + # print err + log.debug("Process finished") + return code, out, err + + def numTables(self): + return self.tables + + def shortDescription(self): + return 'Creates %d tables and then deletes them. '\ + 'Lower score is better.' % (self.numTables()) + + def setSpeed(self, speed): + if speed == "slow": + self.tables = 50 + elif speed == "medium": + self.tables = 10 + elif speed == "fast": + self.tables = 5 + + def needsAuthentication(self): + return 1 diff --git a/test/bench/lib/IngestBenchmark.py b/test/bench/lib/IngestBenchmark.py new file mode 100755 index 00000000..3036c28b --- /dev/null +++ b/test/bench/lib/IngestBenchmark.py @@ -0,0 +1,94 @@ +# 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. + + +import unittest + +from lib import cloudshell +from lib.Benchmark import Benchmark +from lib.tservers import runEach, tserverNames +from lib.path import accumulo, accumuloJar +from lib.util import sleep +from lib.options import log + +class IngestBenchmark(Benchmark): + "TestIngest records on each tserver" + + rows = 1000000 + + def setUp(self): + code, out, err = cloudshell.run(self.username, self.password, 'table test_ingest\n') + if out.find('does not exist') == -1: + log.debug("Deleting table test_ingest") + code, out, err = cloudshell.run(self.username, self.password, 'deletetable test_ingest -f\n') + self.assertEquals(code, 0, "Could not delete the table 'test_ingest'") + code, out, err = cloudshell.run(self.username, self.password, 'createtable test_ingest\n') + self.assertEqual(code, 0, "Could not create the table 'test_ingest'") + Benchmark.setUp(self) + + def tearDown(self): + command = 'deletetable test_ingest -f\n' + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, "Could not delete the table 'test_ingest'") + Benchmark.tearDown(self) + + def size(self): + return 50 + + def random(self): + return 56 + + def count(self): + return self.rows + + def runTest(self): + commands = {} + for i, s in enumerate(tserverNames()): + commands[s] = '%s %s -u %s -p %s --size %d --random %d --rows %d --start %d --cols %d' % ( + accumulo('bin', 'accumulo'), + 'org.apache.accumulo.test.TestIngest', + self.username, self.password, + self.size(), + self.random(), + self.count(), + i*self.count(), + 1) + results = runEach(commands) + codes = {} + for tserver, (code, out, err) in results.items(): + codes.setdefault(code, []) + codes[code].append(tserver) + for code, tservers in codes.items(): + if code != 0: + self.assertEqual(code, 0, "Bad exit code (%d) from tservers %r" % (code, tservers)) + + def score(self): + if self.finished: + return self.count() * self.size() / 1e6 / self.runTime() + return 0. + + def shortDescription(self): + return 'Ingest %d rows of values %d bytes on every tserver. '\ + 'Higher is better.' % (self.count(), self.size()) + + def setSpeed(self, speed): + if speed == "fast": + self.rows = 10000 + elif speed == "medium": + self.rows = 100000 + elif speed == "slow": + self.rows = 1000000 + diff --git a/test/bench/lib/RowHashBenchmark.py b/test/bench/lib/RowHashBenchmark.py new file mode 100755 index 00000000..34009d64 --- /dev/null +++ b/test/bench/lib/RowHashBenchmark.py @@ -0,0 +1,136 @@ +# 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. + +import unittest + +import subprocess +import os +import glob +import random +import time + +from lib import cloudshell, runner, path +from lib.Benchmark import Benchmark +from lib.tservers import runEach, tserverNames +from lib.path import accumulo, accumuloJar +from lib.util import sleep +from lib.options import log + +class RowHashBenchmark(Benchmark): + "RowHashing Benchmark" + + keymin = 10 + keymax = 10 + valmin = 80 + valmax = 80 + rows = 1000000 + maxmaps = 60 + hadoop_version = '' + input_table = 'RowHashTestInput' + output_table = 'RowHashTestOutput' + + def setUp(self): + dir = os.path.dirname(os.path.realpath(__file__)) + file = os.path.join( dir, 'splits' ) + code, out, err = cloudshell.run(self.username, self.password, 'table %s\n' % self.input_table) + if out.find('does not exist') == -1: + code, out, err = cloudshell.run(self.username, self.password, 'deletetable -f %s\n' % self.input_table) + self.sleep(15) + code, out, err = cloudshell.run(self.username, self.password, "createtable %s -sf %s\n" % (self.input_table, file)) + code, out, err = cloudshell.run(self.username, self.password, 'table %s\n' % self.output_table) + if out.find('does not exist') == -1: + code, out, err = cloudshell.run(self.username, self.password, 'deletetable -f %s\n' % + self.output_table) + self.sleep(15) + code, out, err = cloudshell.run(self.username, self.password, "createtable %s -sf %s\n" % (self.output_table, file)) + command = self.buildcommand('org.apache.accumulo.test.mapreduce.TeraSortIngest', + '--count', self.numrows(), + '-nk', self.keysizemin(), + '-xk', self.keysizemax(), + '-nv', self.minvaluesize(), + '-xv', self.maxvaluesize(), + '--table', self.input_table, + '-i', self.getInstance(), + '-z', self.getZookeepers(), + '-u', self.getUsername(), + '-p', self.getPassword(), + '--splits', self.maxmaps) + handle = runner.start(command, stdin=subprocess.PIPE) + log.debug("Running: %r", command) + out, err = handle.communicate("") + Benchmark.setUp(self) + + def tearDown(self): + code, out, err = cloudshell.run(self.username, self.password, "deletetable -f %s\n" % self.input_table) + self.assertEqual(code, 0, 'Could not delete %s, %s' % (self.input_table, out)) + code, out, err = cloudshell.run(self.username, self.password, "deletetable -f %s\n" % self.output_table) + self.assertEqual(code, 0, 'Could not delete %s, %s' % (self.output_table, out)) + Benchmark.tearDown(self) + + def tearDown(self): + code, out, err = cloudshell.run(self.username, self.password, "deletetable %s\n" % self.input_table) + self.assertEqual(code, 0, 'Could not delete %s, %s' % (self.input_table, out)) + code, out, err = cloudshell.run(self.username, self.password, "deletetable %s\n" % self.output_table) + self.assertEqual(code, 0, 'Could not delete %s, %s' % (self.output_table, out)) + Benchmark.tearDown(self) + + def keysizemin(self): + return self.keymin + + def keysizemax(self): + return self.keymax + + def numrows(self): + return self.rows + + def minvaluesize(self): + return self.valmin + + def maxvaluesize(self): + return self.valmax + + def runTest(self): + command = self.buildcommand('org.apache.accumulo.test.mapreduce.RowHash', + self.getInstance(), + self.getZookeepers(), + self.getUsername(), + self.getPassword(), + self.input_table, + 'column:columnqual', + self.output_table, + self.maxmaps) + handle = runner.start(command, stdin=subprocess.PIPE) + log.debug("Running: %r", command) + out, err = handle.communicate("") + log.debug("Process finished: %d (%s)", handle.returncode, ' '.join(handle.command)) + return handle.returncode, out, err + + def shortDescription(self): + return 'Hashes %d rows from one table and outputs them into another table. '\ + 'Lower score is better.' % (self.numrows()) + + def setSpeed(self, speed): + if speed == "slow": + self.rows = 1000000 + self.maxmaps = 400 + elif speed == "medium": + self.rows = 100000 + self.maxmaps = 40 + else: # if speed == "fast" + self.rows = 10000 + self.maxmaps = 4 + + def needsAuthentication(self): + return 1 diff --git a/test/bench/lib/TableSplitsBenchmark.py b/test/bench/lib/TableSplitsBenchmark.py new file mode 100755 index 00000000..2a21fe46 --- /dev/null +++ b/test/bench/lib/TableSplitsBenchmark.py @@ -0,0 +1,76 @@ +# 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. + +import unittest + +import subprocess +import os +import glob +import random +import time + +from lib import cloudshell, runner, path +from lib.Benchmark import Benchmark +from lib.tservers import runEach, tserverNames +from lib.path import accumulo, accumuloJar +from lib.util import sleep +from lib.options import log + +class TableSplitsBenchmark(Benchmark): + "Creating a table with predefined splits and then deletes it" + + splitsfile = 'slowsplits' + tablename = 'test_splits' + + def setUp(self): + # Need to generate a splits file for each speed + code, out, err = cloudshell.run(self.username, self.password, 'table %s\n' % self.tablename) + if out.find('does not exist') == -1: + log.debug('Deleting table %s' % self.tablename) + code, out, err = cloudshell.run(self.username, self.password, 'deletetable %s -f\n' % self.tablename) + self.assertEqual(code, 0, "Could not delete table") + Benchmark.setUp(self) + + def runTest(self): + command = 'createtable %s -sf %s\n' % (self.tablename, self.splitsfile) + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, 'Could not create table: %s' % out) + return code, out, err + + def shortDescription(self): + return 'Creates a table with splits. Lower score is better.' + + def tearDown(self): + command = 'deletetable %s -f\n' % self.tablename + log.debug("Running Command %r", command) + code, out, err = cloudshell.run(self.username, self.password, command) + self.assertEqual(code, 0, "Could not delete table") + log.debug("Process finished") + Benchmark.tearDown(self) + + def setSpeed(self, speed): + dir = os.path.dirname(os.path.realpath(__file__)) + if speed == "slow": + splitsfile = 'slowsplits' + elif speed == "medium": + splitsfile = 'mediumsplits' + else: # speed == "fast" + splitsfile = 'fastsplits' + self.splitsfile = os.path.join( dir, splitsfile) + + def needsAuthentication(self): + return 1 + diff --git a/test/bench/lib/TeraSortBenchmark.py b/test/bench/lib/TeraSortBenchmark.py new file mode 100755 index 00000000..f9984b26 --- /dev/null +++ b/test/bench/lib/TeraSortBenchmark.py @@ -0,0 +1,110 @@ +# 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. + +import unittest + +import subprocess +import os +import glob +import random +import time + +from lib import cloudshell, runner, path +from lib.Benchmark import Benchmark +from lib.util import sleep +from lib.options import log + +class TeraSortBenchmark(Benchmark): + "TeraSort in the cloud" + + keymin = 10 + keymax = 10 + valmin = 78 + valmax = 78 + rows = 10000000000 + numsplits = 400 + # Change this number to modify how the jobs are run on hadoop + rows_per_split = 250000 + hadoop_version = '' + tablename = 'CloudIngestTest' + + + def setUp(self): + code, out, err = cloudshell.run(self.username, self.password, 'table %s\n' % self.tablename) + if out.find('does not exist') == -1: + log.debug('Deleting table %s' % self.tablename) + code, out, err = cloudshell.run(self.username, self.password, 'deletetable -f %s\n' % self.tablename) + Benchmark.setUp(self) + + def tearDown(self): + code, out, err = cloudshell.run(self.username, self.password, "deletetable -f %s\n" % self.tablename) + self.assertEqual(code, 0, 'Could not delete %s, %s' % (self.tablename, out)) + Benchmark.tearDown(self) + + def keysizemin(self): + return self.keymin + + def keysizemax(self): + return self.keymax + + def numrows(self): + return self.rows + + def minvaluesize(self): + return self.valmin + + def maxvaluesize(self): + return self.valmax + + def runTest(self): + dir = os.path.dirname(os.path.realpath(__file__)) + file = os.path.join( dir, 'splits' ) + code, out, err = cloudshell.run(self.username, self.password, "createtable %s -sf %s\n" % (self.tablename, file)) + command = self.buildcommand('org.apache.accumulo.test.mapreduce.TeraSortIngest', + '--count', self.numrows(), + '-nk', self.keysizemin(), + '-xk', self.keysizemax(), + '-nv', self.minvaluesize(), + '-xv', self.maxvaluesize(), + '-t', self.tablename, + '-i', self.instance, + '-z', self.zookeepers, + '-u', self.username, + '-p', self.password, + '--splits', self.numsplits) + handle = runner.start(command, stdin=subprocess.PIPE) + log.debug("Running: %r", command) + out, err = handle.communicate("") + log.debug("Process finished: %d (%s)", handle.returncode, ' '.join(handle.command)) + self.assertEqual(handle.returncode, 0, "Job did not complete successfully") + return handle.returncode, out, err + + def needsAuthentication(self): + return 1 + + def shortDescription(self): + return 'Ingests %d rows (to be sorted). '\ + 'Lower score is better.' % (self.numrows()) + + def setSpeed(self, speed): + if speed == "slow": + self.rows = 10000000000 + self.numsplits = 400 + elif speed == "medium": + self.rows = 10000000 + self.numsplits = 40 + elif speed == "fast": + self.rows = 10000 + self.numsplits = 4 diff --git a/test/bench/lib/__init__.py b/test/bench/lib/__init__.py new file mode 100755 index 00000000..09697dce --- /dev/null +++ b/test/bench/lib/__init__.py @@ -0,0 +1,15 @@ +# 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. + diff --git a/test/bench/lib/cloudshell.py b/test/bench/lib/cloudshell.py new file mode 100755 index 00000000..8c552fbe --- /dev/null +++ b/test/bench/lib/cloudshell.py @@ -0,0 +1,33 @@ +# 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. + + +import subprocess + +from lib import path +from lib import runner +from lib.options import log + + +def run(username, password, input): + "Run a command in accumulo" + handle = runner.start([path.accumulo('bin', 'accumulo'), 'shell', '-u', username, '-p', password], + stdin=subprocess.PIPE) + log.debug("Running: %r", input) + out, err = handle.communicate(input) + log.debug("Process finished: %d (%s)", + handle.returncode, + ' '.join(handle.command)) + return handle.returncode, out, err diff --git a/test/bench/lib/fastsplits b/test/bench/lib/fastsplits new file mode 100644 index 00000000..63a31260 --- /dev/null +++ b/test/bench/lib/fastsplits @@ -0,0 +1,300 @@ + 8_[[@]V)g +! Hm=E#(X} +!N<~+ Y7LW +"!#02~UMAc +"C[`\+0<6_ +#$;~d/|&`) +#)S(BIkV+8 +#9||w=nFUS +#, +$l"]2vq>x% +$oBIw^3Q6g +%+QoBOq:F[ +%6hnyeW+xu +%9Ty&[2+AR +%Bu+%{xQk} +%IETwl`7@u +%NH>"34"i +,4O]Y@h-%. +,A_;2J/poG +,j.9UR]}J@ +,xWmj0-)'fXk +4G3;-_4c@z +4_e1LVoOd{ +4d[Bz~I#*c +4h-S][gEg@ +4p,Z_@(F(0 +5r8Qtn]%Zp +6%~Wgz .`r +6@vYDc%@-k +6UB2tRz9VG +6]NN,?mXfF +6|;OY,eC^m +71NFCWfkIg +7T,N*L0|B$ +7X&k)Mnw7A +7hu(z$ .d2 +7|m#)G;g7' +8 gNp8#@5R +8$/O8ZhCMu +;E`6m/n=B( +;eT#S*'^m@ +;~9@/]L7t4 +<+`jK~}h[K +32ir!c?v) +>H@1?E 1Sr +>IyA_El%$K +>rL,qr~TqI +?%IETwl`7@ +?g8{KnWc(2 +?lQTa c1DE +@+@B*+,%9E +@BT^o`(~GO +@`,Y#9tX0V +A-Xo.RbIa3 +A7hu(z$ .d +AFV1^w+wyf +A_;2J/poG# +Aa|rL,qr~Tq +BFcq5=qK[! +BbY^)[9#jl +B|~j!/o.AX +C3!##k\4+L +C8 gNp8#@5 +C876IhCMu< +HQ\jNVWxb +H]>129!z1Z +Hm#7?^2*^v +IBHd-yE=*8 +Id~_^eb*:~ +IyA_El%$KO +J2j^@_uGQE +KA!~Bd6{w/ +KgIM-K=cKv +L(D\MVWT]q +Lb4`Xe|Ide +M:F.DT"MrX +ME746NYvO} +METKC~VFEZ +MHX\>VSx"a +MQ_,z sLfk +N;m+QQ)MM: +Nk"~w?=7fW +O$oBIw^3Q6 +O7h/k3y4Wx +O9L>=;KJo? +O32ir!c?v +QQ;5Zp=+%\ +QbIzMD=/~) +Qj{.TkiD2f +RH{]O><_tb +Rw2v5>R*~X +S)]6byA mX +Sf@jpK'i:? +T7|m#)G;g7 +TlP76bt0.d +U=n~3gOx(g +U>7^ux VgB +UW^?k\EMo| +UmI2_.}qy} +VA>Shg;h$Q +VHFQ0J%G]u +Vxd,Vz>!t] +WVxd,Vz>!t +W^?k\EMo|g +X%u=$fKWE# +X_RMVKC8/K +Xa!\`%>};~ +Xq'a@Ve@LH +Xy1"}W; +[S)]6byA m +[Sb.8r[b5Q +[VD4|Swvl] +[[Lx&Ubo_& +\aZ{Kux;oE +]#j.mUz +_*u^.u0F9Dw +`uBR|..b#] +a$`"WiH2o> +b4`Xe|IdeQ +bIzMD=/~)b +bpip1V|XZ# +c1gueA\riS +d6w&X@GAFQ +d[9VQ2>"}W +d[Bz~I#*c? +dqX7xL$:BS +f@jpK'i:?H +fZn),"ibhj +gCs5Xj,Nw> +gz|'Kxfm2O +h$VdF;\Xyx +i]C)h.:Q*o +iv@PFpmW9. +jY~[[%5OX@ +jjYSis[@q0 +jmRxiM\A-a +j{.TkiD2f9 +kPgh4d,q@W +k[[Lx&Ubo_ +l"]2vq>x%| +lP76bt0.dq +lkj=KxI!)# +m#7?^2*^vB +m6Qf=5KHMx +m@+@B*+,%9 +mRxiM\A-a, +mzCemV4I]w +nmb]2LQMD0 +no79`(x]f% +oJ'&t#@~ty +p"C[`\+0<6 +p%!]7$j.mU +pip1V|XZ#* +ptinhidK?. +q7X&k)Mnw7 +qB|~j!/o.A +qDzPRZ5}O* +qX7xL$:BS& +rC3!##k\4+ +rXa!\`%>}; +ry[fI-whv9 +t9l&/-RrdC +t;E`6m/n=B +tw+$P@p7** +uBR|..b#]X +u~]KWLu s% +vEU,y{4\Q7 +w+$P@p7**t +wD+*d~OplV +wIBHd-yE=* +wOwA\-ko3, +wq2~T%*`C. +xL4\6T(T8z +xWmj0-)VSx" +{i_V5T0Y~ +|;OY,eC^mU +|[VD4|Swvl +}/tlA^J7dS +}1mj^DS'%i + mX(H;1tZh + nvsr*n7Nm +! Hm=E#(X} +!%-3jzL^Vx +!3!.b\Q,:5 +!97l@xN m' +!R?OI;j7fp +!`2TDk_[m} +!dZ)Kqr\Z$ +!dpSG}M^[! +!e;ep_3A +"C[`\+0<6_ +"S$kPgh4d, +"h4<#+*5^} +"iw\o7oX`^ +#$;~d/|&`) +#+*5^}O5h5 +#9||w=nFUS +#, +$oBIw^3Q6g +${bV80R>13 +%!]7$ASN;z +&|?\H&?#d\ +'&D%F6V"t) +':X}rq!g7v +'pXQ1sTGoA +'tx\r-?j^@ +'z@KNK@<=, +($raW"A^%n +(2]N;m+QQ) +(O2Z+[\fEb +(U2mqCC|fy +(X}_ZT[-~u +(Y+Hm#7?^2 +(aK`!?\~;o +(qL0nP1qAM +)#K;~9@/]L +)S(BIkV+8$ +)UMqX3#~E1 +)g~h'DpOn9 +)ue+Y%*X|x +*$>>"34"iw +*$GrC3!##k +*7ozJGKA@~ +*DUx9EMa[? +*WUm48!G[? +*c?m@+@B*+ +*e>=+sqj~[ +*mZ>~yNm:2 +*qVlW` BEy +*u^.u0F9Dw` ++8$71NFCWf ++C!GtinpGM ++Mq\;3:Zn~ ++QoBOq:F[7 ++sqj~[8>68 +/BB]uzzf.g +/KVyCe3bHH +/OB^&Z.\,w +/^>ku2$G$s +0$\:M,0GbK +0&}~-jwCbA +0'Z[m^>Cxy +0Q:S5|;s y +0e'$+/'Z6L +0w0'7`l]'L +1h +43vu4f@d]] +48!G[?/>Jv +4FdT,Cf`o& +4G3;-_4c@z +4O]Y@h-%.~ +4[s+a$lgN' +4h-S][gEg@ +4jqapc^pKf +4u&:,[%{/M +5!z{MHX\>V +5h5_5}Q8 Y +5qi[8O4c~g +5z'|fA +6^!Z3(vbR@ +6hnyeW+xuj +6iS1$M1ybT +6u`E8_HV#! +6w&X@GAFQ' +7@uA7hu(z$ +7F>0Vs>xU3 +7L)S@SUo@x +7T,N*L0|B$ +7UFT5@Y5qi +7X&k)Mnw7A +7dSvEU,y{4 +7hu(z$ .d2 +7p%d_S8+7b +7w.M`P'VRF +7|m#)G;g7' +8 gNp8#@5R +8$/O8ZLBq:\ +;2I4v0/X!a +;E`6m/n=B( +;m+QQ)MM:z +<+`jK~}h[K +<5eKO/BtO+ +32ir!c?v) +>E|u|~bdnH +>H@1?E 1Sr +>Jv3*mZ>~y +>jMk(GRv5F +>rL,qr~TqI +?6dax-R-c2 +?:)y6{O[UH +?;a)g)>k|I +?h]XP;,'&D +?ky$X1@K|A +?v)wD+*d~O +@)f` +@+@B*+,%9E +@5RjY~[[%5 +@;3ByWA I: +@CN]@)f`m +@WL]H&NywA +@`,Y#9tX0V +@xN m's*qV +A_;2J/poG# +Ar9oCA'~v +A~kBVEU?Jf +BFcq5=qK[! +BT^o`(~GO< +BbY^)[9#jl +B|~j!/o.AX +C.K%6hnyeW +C3!##k\4+L +CDIlg%Lsuu +CZu4y= 1]# +CdWn]fP!g` +Ce3bHH1*$G +CtBJFLcXs= +C{;F7F>0Vs +D+*d~OplV3 +D9>"3 +DGs$+Mq\;3 +DUY`+0W:>F +Dw`oJ'&t#@ +Dz;CM_w~kw +DzPRZ5}O*o +E746NYvO}K +EE*wmHrS_A +EU,y{4\Q7y +EV7'rWEGvb +E]G~:ATEE* +Eg@p-Z6]ka +EhYZ1xvhKV +EypWN*I4D_ +F*[{e[,IG5 +F6V"t)bOCb +FJlUHUI0$\ +FQ'xL4\6T( +FUS9ZL)eI5 +FV1^w+wyfX +Fh9*129! +GQE/UmI2_. +GhZ#W &4Fd +GudpcexeTj +GvjI/BB]uz +G}M^[!3R[k +H&NywA&UO, +H*d.B,one* +H;1tZhH5!z +H=ihzZ83L$ +HFQ0J%G]u> +HMx[S)]6by +HOBc>hCMu< +HQ\jNVWxb +HbF,_LWF}P +I)[)4h-S][ +I1htFb#Yk@ +I=.pMm$]&| +I@DQd0AYRu +IBHd-yE=*8 +IG51qSR$f[ +IMKmf2SH:r +IS+10Q:S5| +IZq^R;B`t7 +IgLLb4`Xe| +IyA_El%$KO +J'&t#@~tyq +J-CQw03`he +J2j^@_uGQE +J4hkmD /)y +JGKA@~/py) +J__e6iS1$M +K7DM&*6`e8 +KA!~Bd6{w/ +K[!O$oBIw^ +Kdn^X%#D +LWfwq2~T%* +M*Hpr,LY5. +M,0GbK=~2X +M,L?h"$^ < +M:z0t!#F.j +METKC~VFEZ +MHX\>VSx"a +MQ_,z sLfk +MR]|XC+9~' +M_w~kw[7L) +Mc'fXkv +Mqaze|RZ=' +Mu=;KJo? +O<_tb +RN{k3uqof\ +RY5k[ndICZ +R[k >E|u|~ +Rev{v$&"XY +Rw2v5>R*~X +S)]6byA mX +S>k>0?5z%a +Sq:|?ky$X1 +SvgC8 gNp8 +T6pAa|A{CZu +U/50bXa7^ux VgB +UO,}1 +VA>Shg;h$Q +Vic_v.LIoG +Vxd,Vz>!t] +W69cl*|{=r +WEbs?(S[Es +WR=\\fgZv_ +W^?k\EMo|g +W` BEyY{r[ +X%#};~ +XkvC876I"}W; +[Sb.8r[b5Q +[UHHipTB`w +[VD4|Swvl] +[[Lx&Ubo_& +[m}y$]d9-3 +[weO=:KhB! +[}f=3qPviV +\Z$_q{7`8Y +\~7Qq)>`%d +]&|9voa'5X +]30hzA}9m: +]7)'VB#}A# +]7wVp5%c? +]<%w`M9[aD +]>129!z1Zv +]@Tw&|?\H& +]Hhe_Z,:?+ +]NN,?mXfFn +]_n27'F{sr +]sr@>j.mUz +]u>5r8Qtn] +]y !N<~+ Y +^ W +`{Y{`r7#i5 +a3{ChOeh,b +aF[|z]5!dp +aZ{Kux;oE3 +a[?06^!Z3( +a^T2=Mo\my +ark:j*[]30 +a|32ir! +g8{KnWc(2] +gCs5Xj,Nw> +gsMz}1ODz; +gz|'Kxfm2O +h-AQ~v3A,4 +hB!m3!Z[JF +hOeh,bJ]y +h[KY%NHrL,qr +ix%| +l?h,4O]Y@h +lP76bt0.dq +lQTa c1DEK +lV3}8G]KhZ +lvOc> 9p7{ +lvh@i+[~8} +ly'y8uN `l +m#7?^2*^vB +m6Qf=5KHMx +m9uz7UFT5@ +m:26{=K@GZ +m>:0/\2>cj +m@8?g8{KnW +mHrS_AsL%b +mRxiM\A-a, +mZy=!@=!e; +m^>CxyxM*H +m^Eh,K5]@T +nc#U4|$K=& +nc)#HF)kl# +ne*3Mc' +nmb]2LQMD0 +no79`(x]f% +o3,d[9VQ2> +o7oX`^v6[S +oE0tUg 'pX +oE38^OB&Sn +oG#$l"]2vq +oW5FdrutGT +of\{K7DM&* +oj[T4Uld,_ +p"fvT$Hm9u +p,Z_@(F(0: +pGM0trHE&k +pKfxaF[|z] +p]i)u]_Zm* +pip1V|XZ#* +ptinhidK?. +py)b]_n27' +q2~T%*`C.K +qR2tI1htFb +qSR$f[7lvh +qX7xL$:BS& +qy}ME746NY +q{7`8Y(MR] +r'Fa3n!Me< +r,LY5.ekCy +r8Qtn]%Zp| +rWEGvb{J__ +r\%R?;a)g) +rc>hRN{k3u +rq!g7v4*WU +s yAfZn)," +sYFUS>k>0? +sof,OQOSq: +s{WIAn~d8B +t!#F.jycv% +t4%\aZ{Kux +t9l&/-RrdC +t<)4;e}n.Z +tGTjAr9oCA +tGXi"?3!=5 +tO+1Op!:AI +trHE&kdPDM +twZ}"9"8hg +tyq'H*d.B, +u2$G$s&rc> +u4Ag15R0e' +u?@J)ue+Y% +uBR|..b#]X +uQ3g!:{9M# +uRyS= +u]_Zm*X*7o +v$&"XY[}xC +v.LIoG'-#J +v3lFW:Ih{Q +v7/VXuP= C +vBxU"pSST4 +voa'5X-&b/ +w )%Ar9u?@ +w+$P@p7**t +w7AQQ;5Zp= +w?:@7O%*e> +wOwA\-ko3, +w|@-}e+G}} +x"a'Fh9*W[Yb+Znp]i +xujzP}sUVQ +y1 + clA)/1,o0 + p'NiRfrp3 +! Hm=E#(X} +!%-3jzL^Vx +!&P1xnSIJc +!'>p]D)_fQ +!/\$4oMrW& +!/o.AXX_RM +!0f2rL6@Bu +!:{9M#+/OB +!?\~;oli.M +!@=!e;QhL~r& +"9"8hgssof +"?3!=5lHbF +"BN/#%-lA1 +"C[`\+0<6_ +"\AhQ4?"WL +"l#n6jM +#!&]7)'VB# +#!Fv7/VXuP +#$;~d/|&`) +#%-lA18kw' +#+*5^}O5h5 +#9||w=nFUS +#;lk?rXU/5 +#a +#k\4+L:HOB +#pCWR=\\fg +#x97^QGxAN +$, +$j,S@*vqo` +$nT`dP4-]k +$oBIw^3Q6g +$o~Fdu,ljs +$s&rc>hRN{ +%!]7$ +%n/R&:jtXw +&*6`e8p*DU +&A'N\FIZXq +&CrH4,|c/6 +&L#.Jmp(DR +&QaBTSZBI` +&Z.\,wJ!dZ +&_2O[cq4ep +&kdPDMJ43v +&srQw5aK}Y +&|?\H&?#d\ +'"g:ya_{_$ +'"wq%n/R&: +'F=_G!Id~_ +'LW6u`E8_H +'Q b*WP\s, +'mrppzMh4+ +'tx\r-?j^@ +($raW"A^%n +('LV=7D#Bo +(;!P4F'2`{ +(GRv5FyD9< +(J"50*R;o< +(RE::l24[s +(Tb*w93B=q +(U2mqCC|fy +(Xb>)$^@D( +(iJ^e/()`b +)*D\ !cqon +)6l-2u}&P} +)Mnw7AQQ;5 +)S(BIkV+8$ +)UMqX3#~E1 +)g~h'DpOn9 +)ue+Y%*X|x +)y%1J"oyVz +*$>>"34"iw +*'^m@8?g8{ +*DUx9EMa[? +*H:Z -0O5= +*Hte \@cxR +*K$9[UZ32; +*WP\s,*D.d +*mZ>~yNm:2 +*p8LEas6=f +*uI6v0x%sW +*u^.u0F9Dw` ++C!GtinpGM ++Mq\;3:Zn~ ++QoBOq:F[7 ++[\fEbn{|N ++gu=312H=i ++sqj~[8p]D +,o0=3E:4#@ +,qosnDyu]. +-$O;nf/dKo +-){&T;kH|/ +-34zl005z' +-: VPO@M# +-FPWS=J@\{ +-SMJ0^zt<) +-T86yB|B#" +-Xo.RbIa3{ +-Z6]ka+Omr +-jwCbA9sYF +.)pDs_!O`K +.YJ|irdw_@ +.Zncs5|IqC +.^g$7rd=Y* +.jycv%/y_@ +.l5S`7lzv3 +/-,5R|f1$u +/-@cf.uJ-C +/BB]uzzf.g +/NI2R`i"5) +/OB^&Z.\,w +/\2>cj,b+' +0%36h!Wv]5 +0*5[Ue0Tgj +0*R;oZDIq +2>"}W;qDzP +2F~b"l#n6< +2Pj89$_1uU +2WQO@6PDGs +2`{pN[df,~ +2tacz_=}.6 +3!Z[JF1EV7 +3(vbR@hoE0 +312H=ihzZ8 +3Jv +48l7w.M`P' +4G3;-_4c@z +4KRGqCQ9W^ +4O]Y@h-%.~ +4Uld,_Eu4A +4X?69VB!Go +4epwLxsb=x +4f@d]]L^!C +4h-S][gEg@ +4jqapc^pKf +4u&:,[%{/M +4z5Tl+,b^M +4|$K=&7[(5 +5.ekCy74jq +5@Y5qi[8O4 +5FyD9AS +6=Fq,}AUJe +6@vYDc%@-k +6Lt{|6D<5e +6O o/-@cf. +6Q~=v;8!jo +6UB2tRz9VG +6^!Z3(vbR@ +6hnyeW+xuj +6iS1$M1ybT +6rUuUNlC#r +6tvW2+>O#2 +6w&X@GAFQ' +6wp$Br(.mo +6{O[UHHipT +7'F{srd+C! +74(E0kxTv+ +7961a/r%'f +7>np]i)u]_ +7F>0Vs>xU3 +7O%*e>=+sq +7T,N*L0|B$ +7UFT5@Y5qi +7X&k)Mnw7A +7^ddJlgwL< +7`l]'LW6u` +7bNEhYZ1xv +7hu(z$ .d2 +7v4*WUm48! +7{A nvsr*n +7|m#)G;g7' +8 gNp8#@5R +8!l4CN0&L# +8$/O8Z13_IZq +876I$a +8G]KhZ=^vD +8LPAPHi$*~ +8Lsl=hpm!/ +8O4c~gyV<2 +8P}k#>EEI= +8X~QuCBfmG +8Y(MR]|XC+ +8_HV#!&]7) +8_[[@]V)gn +8`32 +9RLf@z>!3! +9Sd-y<;j{3 +9hn+-p''"g +9zw"sCJ,mH +:%7/^mI#*^ +:,7c#E@=cF +:.`u5Z=<9e +:5eO^1mcl +:>a{hK-d-^ +:ATEE*wmHr +:F.DT"MrX= +:K*2>D#nc# +:\e{?Gv=pT +:l24[s+a$l +:r\*zQA'"w +;3:Zn~2twZ +;E`6m/n=B( +;e}n.Zncs5 +;m+QQ)MM:z +;oli.MH<0h +;omC6rUuUN +;zqNrRPUS/ +<+`jK~}h[K +<0h0C?-ly' +<5eKO/BtO+ +ku2$ +32ir!c?v) +>ASN;zqNrR +>A{CZu4y= +>D#nc#U4|$ +>E|u|~bdnH +>F9Dw`oJ'& +>H@1?E 1Sr +>LBq:\e{?G +>ep_3A0IMK +>hCMul&{f3F;n~ +>qPviV]A~k +>rL,qr~TqI +? u".#b>ep +?(S[Es"yoJ +?.w~XEWayx +?:)y6{O[UH +?;a)g)>k|I +?bJ*f;>Clb +?h]XP;,'&D +?ky$X1@K|A +?l\9>l +@)f` +@+@B*+,%9E +@6PDGs$+Mq +@GAFQ'xL4\ +@SUo@x8 >m +@`,Y#9tX0V +@h-%.~VHFQ +@qI82}DU;5 +@x8 >mj^DS +@xN m's*qV +@z>!3!.b\Q +@~/py)b]_n +A#VW69cl*| +AIE@CN]@)f +AMD\Tv*>qP +A_;2J/poG# +Ah7ZH`^CT] +An~d8B563Q +Ar9oCA'~v +Ar9u?@J)ue +B'}L;XR*Fx +B,one*3Mc< +BFcq5=qK[! +BOD nfLqfC +BT^o`(~GO< +BbY^)[9#jl +Br(.mo[d'q +B|~j!/o.AX +C#r:VhD45= +C3!##k\4+L +C>nWUbKv!) +C?-ly'y8uN +CA'~v 'FJl +CV:Mqaze|R +CZ]L2t?(RE +C] @Y8=?.w +C_+L!/\$4o +CdWn]fP!g` +Ce3bHH1*$G +ClS)g\fcsM +Clb2QlYLv! +CtBJFLcXs= +C~StqP]"\5 +D+*d~OplV3 +D.dwBOD nf +DQ2Ps|C@Cy +Dk_[m}y$]d +DxJYjgIr?l +DzPRZ5}O*o +D~CY#!Fv7/ +E&l/#Udir> +E6/Z_Gf1K5 +E746NYvO}K +E +HH1*$GrC3! +HOBc>hCMu< +HP%9xjE,+v +HQ\jNVWxb +HR$Bs_f|Fa +HUI0$\:M,0 +HXnm>:0/\2 +HYhv@qI82} +HdM3gL 9-W +H|5r8Q +J'&t#@~tyq +J-CQw03`he +J.H3#pCWR= +J2j^@_uGQE +JD9l +JGKA@~/py) +Jfs@L97DQ2 +Jh+9i^,r7i +JlK,}~/+gu +Jmp(DR&[N3 +K$Rw[.AWEb +K7DM&*6`e8 +KA!~Bd6{w/ +KC8/KVyCe3 +KCLqPv-'XH +KEyb-R_#;l +KV*':X}rq! +K[2"A]]ZiC +Kc([VE!;'l +KgIM-K=cKv +Kqr\Z$_q{7 +L a^/4U5Vz +L$R(O2Z+[\ +L(D\MVWT]q +L4\6T(T8z/ +L@SfIJQ7p% +Lgv{|N&Zp' +Lxsb=x}jVSx"a +MQ_,z sLfk +M_w~kw[7L) +Mc'fXkv +Mm$]&|9voa +N'eL+%AFX6 +N5z b*:Kdn +N<~+ Y7LWf +NK@<=,NjP^ +NZA[HXnm>: +N[df,~>uQ3 +Nk"~w?=7fW +Nm8LMwq:K* +Np+U~LT89^ +NvQ>S=li=;KJo? +Ot +QlYLv!c!tf +R-,u-#4>E` +R/%vW7Bs:2 +R4PErdDVpt +R;B`t7\#L3 +RFuRY5k[nd +RH{]O><_tb +RN{k3uqof\ +RbIa3{ChOe +RoTOC'XGhZ +Rw2v5>R*~X +S)]6byA mX +S=lik>0?5z%a +SG{|+])j-c +SHHCluc@Ub +SR/Ig;CQ5) +SnJ.[4/|hi +T$Hm9uz7UF +T(T8z/lkj= +T23Y{XPXSB +T4J>jMk(GR +TE^]<%w`M9 +TG_tF-dW3) +Tg\A{CZu +TyF#+)W:.` +U#R-DxJYjg +U,KjPMN%[x +U;5"`RLL)Y +U=l^nIk!h, +U=n~3gOx(g +U>7^ux VgB +UCkb('LV=7 +UJe<Shg;h$Q +VB#}A#VW69 +VE!;'lj>LB +VEU?Jfs@L9 +VG'R&_2O[c +VQjDUY`+0W +VU0YB1p%!] +V]]148l7w. +Vci@go:#D; +VhD45=lR4P +Vs>xU3ns{W +Vxd,Vz>!t] +W &4FdT,Cf +W"A^%ngN5z +W%ST^@CJ.I +W-TcJlK,}~ +W1*+}l& +XSBPHR$Bs_ +XSHVy-D`+Y +XY[}xC+6O +Xa!\`%>};~ +Xq'a@Ve@LH +XuP= CL|'` +X|EUa.[@W^ +Y%*X|x-?h] +Y'$wWX{VG' +Y8?yClc0Wb +Yb+ZE| +['>5usQF#C +[,l%-`tW-T +[.AWEbs?(S +[1E;uJ?3dx +[5ELY8?yCl +[9VQ2>"}W; +[?/>Jv3*mZ +[N3RT23Y{X +[Sb.8r[b5Q +[UZ32;i6wp +[VD4|Swvl] +[[Lx&Ubo_& +[]g3~5KmW< +[c5C,+wIKa +[ndICZ]L2t +[pB(aK`!?\ +[t om:4ln8 +[weO=:KhB! +\-ko3,d[9V +\FIZXq;}m~ +\asC#)EKE) +\fgZv_d,"7 +\y,l0*5[Ue +\{.+'2fXAh +]#mGudpcex +].'@;Gk0PF +]>129!z1Zv +]Edd6=Fq,} +]Hhe_Z,:?+ +]L7t4%\aZ{ +]N=ZKmNsPE +]NN,?mXfFn +]_n27'F{sr +]sr@>j.mUz +]}N6_l>5N" +]}bI`R46B0 +^!CrsAG=O8 +^3XNx)|`,l +^DS'%izC{; +^OB&SnJ.[4 +^QGxANONZA +^r>]7wVp5% +^}O5h5_5}Q +_9z+e^[Svg +_AsL%bR~/q +_LWF}P-koD +_ON,K_yAxi +_RMVKC8/KV +_S8+7bNEhY +_U`F?bJ*f; +_Z,:?+9[we +_ZKl?h,4O] +_e1LVoOd{| +_fQ"gncbqO +_hS5WUld}1 +_l>5N"_U#R +`%>};~qB|~ +`(~GO1 +`*uX?A^"g2 +`7lzv3/o'| +`M9[aD]{dB +`P'VRFuRY5 +`R46B0m;om +`RLL)Y0"\A +`Z!(qL0nP1 +`^v6[SJTvb +`lI\+][D~C +`r7#i5ux>W +`s*qqm/=SI +`wfRev{v$& +a$lgN'eL+% +aD]{dB0K$R +aF[|z]5!dp +aPtOQM;d U +aX!eG-$q4. +aZ{Kux;oE3 +anH=PW_Gwv +ark:j*[]30 +a|k +bB"Ref)Qe7 +bIzMD=/~)b +bK2XC9Gy7E +bK=~2Xuf`8 +bXa2Pj89$ +d'qE0,gtGX +d([^tNXdQ/ +d0AYRu-mZy +dRo ]RT]}b +dXAG03O".v +d[Bz~I#*c? +drutGTjAr9 +d~_^eb*:~f +e.E;2I4v0/ +e3QY9HX-4k +e6;Vic_v.L +e +e>t"TG_tF- +eT#S*'^m@8 +eW+xujzP}s +e[,IG51qSR +eb*:~fOCtB +e|RZ=']"h4 +f MiN3eN,+ +f2SH:r\*zQ +f3F;n~BlPd +f9,QGZ%n91 +fk|ISgsM +g8{KnWc(2] +gCs5Xj,Nw> +gSadto8ark +gncbqOGUCk +gsMz}1ODz; +gz|'Kxfm2O +h"$^ zTt +i5ux>W[Yb+ +i6N!)lTBG4 +i7?2D8H[5E +iV]A~kBVEU +i]C)h.:Q*o +i^,r7i+&Cr +ipTB`wfRev +ir>>686&z. +iv@PFpmW9. +j*[]30hzA} +j-f#:\rsIr +j2|QJ)&,oq +jx%| +l$g0I*o,G0 +l*|{=rfCxyxM*H +m^Eh,K5]@T +nDyu].=NvQ +nP1qAMD\Tv +nWc(2]N;m+ +n]%Zp|~w?: +nc)#HF)kl# +nmb]2LQMD0 +no79`(x]f% +o&|LK{DoW5 +o'|@TyF#+) +o7oX`^v6[S +oAZ8Eh!L@S +oE0tUg 'pX +oG'-#JmI@D +oW5FdrutGT +odr9:\$Ll# +ogpy/NI2R` +oj[T4Uld,_ +opiiZs#Y3[ +ow1Zc.`1)g +oxVge.E;2I +p"fvT$Hm9u +p#8lvOc> 9 +p,Z_@(F(0: +p5%c? u".# +p=+%\}rXa! +pKG#d=WM`%d@@;3 +q2~T%*`C.K +q75Ps`Q4f' +qM|0@5:}]x +qSR$f[7lvh +qX7xL$:BS& +q]'zEE,'u/ +qe2vBxU"pS +qfC%L a^/4 +qm/=SIV.l5 +qo`>Vci@go +q{7`8Y(MR] +r!c?v)wD+* +r*n7Nm8LMw +r,LY5.ekCy +r-?j^@{{%) +r2XOxr?E]G +r8Qtn]%Zp| +r?lOC] @Y8 +rFl?@t74-O +rW&!Py*Js[ +rWEGvb{J__ +rd>x8(2vWe +rdDVpt!]}N +riWO"%y77} +rq!g7v4*WU +r{iF#i?4[w +sAG=O8noj[ +sIr\ow1Zc. +s_!O`K`F*[ +sof,OQOSq: +s{WIAn~d8B +s|C@CySoxV +t!#F.jycv% +t)bOCbRO@k +t1;{=Oh"ur +t7\#L3P9RL +t9l&/-RrdC +tH'e>x0{?? +tKab`fYpOL +tXwWz@r?6d +tinpGM0trH +trHE&kdPDM +ttDr\%R?;a +twI*IBc2F~ +twZ}"9"8hg +u'J[!&P1xn +u2$G$s&rc> +u4Ag15R0e' +uBR|..b#]X +uQ3g!:{9M# +u]_Zm*X*7o +uj7@l".C_+ +uq|#5{u]#1 +uuC${bV80R +ux;oE38^OB +uzzf.g^!`2 +v 0N4ne -\ +v$&"XY[}xC +v.LIoG'-#J +v0/X!aEJ4h +v3lFW:Ih{Q +v_d,"7lu'J +va1XuCc+*F +vb{J__e6iS +voa'5X-&b/ +w )%Ar9u?@ +w+$P@p7**t +w03`he%m^E +w5aK}YZ#=R +w93B=q'0hQ +w?:@7O%*e> +wLnp]i +xuHrk~a_Y} +xyxM*Hpr,L +y-D`+Y_r{i +y1]7w +yWA I:qh-A +y[fI-whv9^ +y\F]9Klfe +y_@;{hWz?D +ya_{_$|Kc( +z3Z2 a6_eW +z@r?6dax-R +zA}9m:IP +>>>>>>>>>> +?P +?????????? +@P +@@@@@@@@@@ +AP +AAAAAAAAAA +BP +BBBBBBBBBB +CP +CCCCCCCCCC +DP +DDDDDDDDDD +EP +EEEEEEEEEE +FP +FFFFFFFFFF +GP +GGGGGGGGGG +HP +HHHHHHHHHH +IP +IIIIIIIIII +JP +JJJJJJJJJJ +KP +KKKKKKKKKK +LP +LLLLLLLLLL +MP +MMMMMMMMMM +NP +NNNNNNNNNN +OP +OOOOOOOOOO +PP +PPPPPPPPPP +QP +QQQQQQQQQQ +RP +RRRRRRRRRR +SP +SSSSSSSSSS +TP +TTTTTTTTTT +UP +UUUUUUUUUU +VP +VVVVVVVVVV +WP +WWWWWWWWWW +XP +XXXXXXXXXX +YP +YYYYYYYYYY +ZP +ZZZZZZZZZZ +[P +[[[[[[[[[[ +\P +\\\\\\\\\\ +]P +]]]]]]]]]] +^P +^^^^^^^^^^ +_P +__________ +`P +`````````` +aP +aaaaaaaaaa +bP +bbbbbbbbbb +cP +cccccccccc +dP +dddddddddd +eP +eeeeeeeeee +fP +ffffffffff +gP +gggggggggg +hP +hhhhhhhhhh +iP +iiiiiiiiii +jP +jjjjjjjjjj +kP +kkkkkkkkkk +lP +llllllllll +mP +mmmmmmmmmm +nP +nnnnnnnnnn +oP +oooooooooo +pP +pppppppppp +qP +qqqqqqqqqq +rP +rrrrrrrrrr +sP +ssssssssss +tP +tttttttttt +uP +uuuuuuuuuu +vP +vvvvvvvvvv +wP +wwwwwwwwww +xP +xxxxxxxxxx +yP +yyyyyyyyyy +zP +zzzzzzzzzz +{P +{{{{{{{{{{ +|P +|||||||||| +}P +}}}}}}}}}} +~P +~~~~~~~~~~ diff --git a/test/bench/lib/tservers.py b/test/bench/lib/tservers.py new file mode 100755 index 00000000..b34397a3 --- /dev/null +++ b/test/bench/lib/tservers.py @@ -0,0 +1,89 @@ +# 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. + +import fcntl + +import os +import time +import select +import subprocess + +from lib.path import accumuloConf +from lib.options import log + +def tserverNames(): + return [s.strip() for s in open(accumuloConf('tservers'))] + +def runEach(commandMap): + result = {} + handles = [] + for tserver, command in commandMap.items(): + log.debug("ssh: %s: %s", tserver, command) + handle = subprocess.Popen(['ssh',tserver] + [command], + stdout=subprocess.PIPE, + stderr=subprocess.PIPE) + for h in handle.stdout, handle.stderr: + fcntl.fcntl(h, fcntl.F_SETFL, os.O_NDELAY) + handle.tserver = tserver + handle.command = command + handle.start = time.time() + handles.append(handle) + handlesLeft = set(handles[:]) + while handlesLeft: + fds = {} + doomed = set() + for h in handlesLeft: + more = [] + if h.stdout != None: + more.append(h.stdout) + if h.stderr != None: + more.append(h.stderr) + for m in more: + fds[m] = h + if not more: + doomed.add(h) + handlesLeft -= doomed + if not handlesLeft: break + rd, wr, ex = select.select(fds.keys(), [], [], 10) + for r in rd: + handle = fds[r] + data = r.read(1024) + result.setdefault(handle, ['', '']) + if not data: + if r == handle.stdout: + handle.stdout = None + else: + handle.stderr = None + if r == handle.stdout: + result[handle][0] += data + else: + result[handle][1] += data + if handle.stdout == None and handle.stderr == None: + log.debug("Tserver %s finished in %.2f", + handle.tserver, + time.time() - handle.start) + handle.wait() + if not rd: + log.debug("Waiting on %d tservers (%s...)", + len(handlesLeft), + ', '.join([h.tserver for h in handlesLeft])[:50]) + return dict([(h.tserver, (h.returncode, out, err)) + for h, (out, err) in result.items()]) + +def runAll(command): + tservers = tserverNames() + log.debug("Running %s on %s..", command, ', '.join(tservers)[:50]) + return runEach(dict([(s, command) for s in tservers])) + diff --git a/test/bench/lib/util.py b/test/bench/lib/util.py new file mode 100755 index 00000000..bfe38dbc --- /dev/null +++ b/test/bench/lib/util.py @@ -0,0 +1,20 @@ +# 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. + +import time + + +def sleep(n): + time.sleep(n) diff --git a/test/bench/run.py b/test/bench/run.py new file mode 100755 index 00000000..63a21faa --- /dev/null +++ b/test/bench/run.py @@ -0,0 +1,116 @@ +#! /usr/bin/env python + +# 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. + + +import getopt +import os +import sys +import logging +import unittest + +from lib.options import options, args, log +from lib.Benchmark import Benchmark + +def getBenchmarks(): + import glob + result = [] + here = os.path.dirname(__file__) + sys.path.insert(0, here) + for path in glob.glob('%s/*/*.py' % here): + path = path[len(here):] + if path.find('__init__') >= 0: continue + if path.find('/lib/') >= 0: continue + moduleName = path.replace(os.path.sep, '.') + moduleName = moduleName.lstrip('.')[:-3] + module = __import__(moduleName, globals(), locals(), [moduleName]) + result.extend(list(module.suite())) + return result + +def benchComparator(first, second): + if (first.name() < second.name()): + return -1 + elif (second.name() < first.name()): + return 1 + else: + return 0 + +def main(): + if not os.getenv('HADOOP_HOME'): + print 'Please set the environment variable \'HADOOP_HOME\' before running the benchmarks' + sys.exit(0) + if not os.getenv('ZOOKEEPER_HOME'): + print 'Please set the environment variable \'ZOOKEEPER_HOME\' before running the benchmarks' + sys.exit(0) + if not os.getenv('ACCUMULO_HOME'): + print 'Please set the environment variable \'ACCUMULO_HOME\' before running the benchmarks' + sys.exit(0) + import textwrap + benchmarks = getBenchmarks() + benchmarks.sort(benchComparator) + auth = 0 + for b in benchmarks: + b.setSpeed(options.runSpeed) + if auth == 0 and b.needsAuthentication > 0: + auth = 1 + if options.list: + indent = len(benchmarks[0].name()) + wrap = 78 - indent + prefix = ' ' * indent + ' ' + for b in benchmarks: + desc = b.shortDescription() or "No description" + desc = textwrap.wrap(desc, wrap) + desc = '\n'.join([(prefix + line) for line in desc]) + print '%*s: %s' % (indent, b.name(), desc.lstrip()) + sys.exit(0) + logging.basicConfig(level=options.logLevel) + if auth == 1: + if options.user == '': + print 'User: ', + user = sys.stdin.readline().strip() + else: + user = options.user + if options.password == '': + import getpass + password = getpass.getpass('Password: ') + else: + password = options.password + if options.zookeepers == '': + print 'Zookeepers: ', + zookeepers = sys.stdin.readline().strip() + else: + zookeepers = options.zookeepers + if options.instance == '': + print 'Instance: ', + instance = sys.stdin.readline().strip() + else: + instance = options.instance + Benchmark.instance = instance + Benchmark.zookeepers = zookeepers + Benchmark.instance = instance + Benchmark.password = password + Benchmark.username = user + if args: + benchmarks = [ + b for b in benchmarks if b.name() in args + ] + runner = unittest.TextTestRunner(verbosity=2) + runner.run(unittest.TestSuite(benchmarks)) + for b in benchmarks: + log.info("%30s: %5.2f", b.name(), b.score()) + +if __name__ == '__main__': + main() diff --git a/test/compat/diffAPI.pl b/test/compat/diffAPI.pl new file mode 100755 index 00000000..183655d0 --- /dev/null +++ b/test/compat/diffAPI.pl @@ -0,0 +1,104 @@ +#!/usr/bin/perl + +# 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. + +sub trim($) +{ + my $string = shift; + $string =~ s/^\s+//; + $string =~ s/\s+$//; + return $string; +} + +sub getDeprecated { + my($jar, $class) = @_; + + open(JAVAP, "javap -verbose -public -classpath '$jar' '$class'|"); + + my $lastMethod = ""; + my %deprecated; + + while(){ + chomp(); + if(/^public\s/){ + $lastMethod = $_; + } + if(/Deprecated\:\strue/){ + $lastMethod =~ s/\s+/ /g; + $deprecated{$lastMethod}="true"; + } + } + + close(JAVAP); + + return %deprecated; +} + +sub annotateDeprecated { + my($jar, $class, $deprecated, $outFile) = @_; + open(JAVAP, "javap -public -classpath '$jar' '$class'|"); + open(OUT, ">$outFile"); + my @javapOut = ; + @javapOut = sort(@javapOut); + + for my $line (@javapOut){ + my $trimLine = trim($line); + chomp($line); + $trimLine =~ s/\s+/ /g; + if($deprecated->{$trimLine}){ + print OUT "$line DEPRECATED\n"; + }else{ + print OUT "$line\n"; + } + } + + close(JAVAP); + close(OUT); + +} + +if(scalar(@ARGV) != 2){ + print "Usage : diffAPI.pl \n"; + exit(-1); +} + +$jar1 = $ARGV[0]; +$jar2 = $ARGV[1]; + +$gtCmd = 'egrep "accumulo/core/client/.*class|accumulo/core/data/.*class" | grep -v accumulo/core/client/impl | grep -v accumulo/core/data/thrift | egrep -v "Impl.*class$" | tr / .'; + +@classes1 = `jar tf $jar1 | $gtCmd`; +@classes2 = `jar tf $jar2 | $gtCmd`; + +mkdir("diffWorkDir"); +mkdir("diffWorkDir/jar1"); +mkdir("diffWorkDir/jar2"); + +for $class (@classes1){ + $class = substr($class, 0, length($class) - 7); + %deprecated = getDeprecated($jar1, $class); + annotateDeprecated($jar1, $class, \%deprecated, "diffWorkDir/jar1/$class"); +} + +for $class (@classes2){ + $class = substr($class, 0, length($class) - 7); + %deprecated = getDeprecated($jar2, $class); + annotateDeprecated($jar2, $class, \%deprecated, "diffWorkDir/jar2/$class"); +} + +system("diff -u diffWorkDir/jar1 diffWorkDir/jar2"); +system("rm -rf diffWorkDir"); + diff --git a/test/compat/japi-compliance/README b/test/compat/japi-compliance/README new file mode 100644 index 00000000..0b770506 --- /dev/null +++ b/test/compat/japi-compliance/README @@ -0,0 +1,53 @@ +Title: Java API Compliance Checker Instructions +Notice: 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. + +There is a tool that can analyze the difference between APIs called +japi-compliance[1]. This tool is useful for checking API compatability of +different Accumulo versions. To run this tool edit the xml files to specify +the location of accumulo core jars and set the library version. Then run the +following command. + + japi-compliance-checker.pl -skip-deprecated -old japi-accumulo-1.5.xml -new japi-accumulo-1.6.xml -l accumulo + +Optionally, you can use the --skip-classes argument with the provided exclude_classes.txt file to skip classes from +org.apache.accumulo.core.data that aren't in the public API. + +This directory should have a library configuration file for each release on supported lines as well as an in-progress +for whatever version is currently the master branch. The examples below all make use of version-specific library definitions. + +When looking at a patch release, you should verify that changes introduced are forwards and backwards compatible, per +semver. + + # Backwards compatibility from x.y.z to x.y.(z+1) + japi-compliance-checker.pl -old japi-accumulo-1.6.1.xml -new japi-accumulo-1.6.2.xml -l accumulo --skip-classes=exclude_classes.txt + # Forwards compatibility from x.y.z to x.y.(z+1). Note that the old / new arguments have been swapped. + japi-compliance-checker.pl -new japi-accumulo-1.6.1.xml -old japi-accumulo-1.6.2.xml -l accumulo --skip-classes=exclude_classes.txt + +When looking at a minor release, you should verify that change are backwards compatible, per semver. + + # Backwards compatibility from x.y.z to x.(y+1).0 + japi-compliance-checker.pl -old japi-accumulo-1.6.1.xml -new japi-accumulo-1.7.0.xml -l accumulo --skip-classes=exclude_classes.txt + +When looking at a major release, you should examine removals to make sure they are not capricious. Specifically, you should ensure that +they have been deprecated for a full major version. + + # Advisory backwards compatibility check from x.y.z to (x+1).0.0 + japi-compliance-checker.pl -old japi-accumulo-1.7.0.xml -new japi-accumulo-2.0.0.xml -l accumulo --skip-classes=exclude_classes.txt + +[1]: http://ispras.linuxbase.org/index.php/Java_API_Compliance_Checker + diff --git a/test/compat/japi-compliance/exclude_classes.txt b/test/compat/japi-compliance/exclude_classes.txt new file mode 100644 index 00000000..94356b79 --- /dev/null +++ b/test/compat/japi-compliance/exclude_classes.txt @@ -0,0 +1 @@ +org.apache.accumulo.core.data.KeyValue diff --git a/test/compat/japi-compliance/japi-accumulo-1.5.0.xml b/test/compat/japi-compliance/japi-accumulo-1.5.0.xml new file mode 100644 index 00000000..f49dbb5d --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.5.0.xml @@ -0,0 +1,36 @@ + + + 1.5.0 + + + + CHANGEME/accumulo-1.5.0/lib/accumulo-core.jar + CHANGEME/accumulo-1.5.0/lib/accumulo-minicluster.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.data.thrift + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + + diff --git a/test/compat/japi-compliance/japi-accumulo-1.5.1.xml b/test/compat/japi-compliance/japi-accumulo-1.5.1.xml new file mode 100644 index 00000000..ff92506b --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.5.1.xml @@ -0,0 +1,36 @@ + + + 1.5.1 + + + + CHANGEME/accumulo-1.5.1/lib/accumulo-core.jar + CHANGEME/accumulo-1.5.1/lib/accumulo-minicluster.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.data.thrift + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + + diff --git a/test/compat/japi-compliance/japi-accumulo-1.5.2.xml b/test/compat/japi-compliance/japi-accumulo-1.5.2.xml new file mode 100644 index 00000000..8d7a668e --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.5.2.xml @@ -0,0 +1,36 @@ + + + 1.5.2 + + + + CHANGEME/accumulo-1.5.2/lib/accumulo-core.jar + CHANGEME/accumulo-1.5.2/lib/accumulo-minicluster.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.data.thrift + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + + diff --git a/test/compat/japi-compliance/japi-accumulo-1.6.0.xml b/test/compat/japi-compliance/japi-accumulo-1.6.0.xml new file mode 100644 index 00000000..cb1c22fe --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.6.0.xml @@ -0,0 +1,38 @@ + + + 1.6.0 + + + + CHANGEME/accumulo-1.6.0/core/target/accumulo-core-1.6.0.jar + CHANGEME/accumulo-1.6.0/minicluster/target/accumulo-minicluster-1.6.0.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.client.lexicoders.impl + org.apache.accumulo.core.client.mapreduce.lib.impl + org.apache.accumulo.core.data.thrift + org.apache.accumulo.minicluster.impl + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + diff --git a/test/compat/japi-compliance/japi-accumulo-1.6.1.xml b/test/compat/japi-compliance/japi-accumulo-1.6.1.xml new file mode 100644 index 00000000..30e60689 --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.6.1.xml @@ -0,0 +1,38 @@ + + + 1.6.1 + + + + CHANGEME/accumulo-1.6.1/core/target/accumulo-core-1.6.1.jar + CHANGEME/accumulo-1.6.1/minicluster/target/accumulo-minicluster-1.6.1.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.client.lexicoders.impl + org.apache.accumulo.core.client.mapreduce.lib.impl + org.apache.accumulo.core.data.thrift + org.apache.accumulo.minicluster.impl + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + diff --git a/test/compat/japi-compliance/japi-accumulo-1.6.2.xml b/test/compat/japi-compliance/japi-accumulo-1.6.2.xml new file mode 100644 index 00000000..83272061 --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.6.2.xml @@ -0,0 +1,38 @@ + + + 1.6.2 + + + + CHANGEME/accumulo-1.6.2/core/target/accumulo-core-1.6.2.jar + CHANGEME/accumulo-1.6.2/minicluster/target/accumulo-minicluster-1.6.2.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.client.lexicoders.impl + org.apache.accumulo.core.client.mapreduce.lib.impl + org.apache.accumulo.core.data.thrift + org.apache.accumulo.minicluster.impl + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + diff --git a/test/compat/japi-compliance/japi-accumulo-1.7.0.xml b/test/compat/japi-compliance/japi-accumulo-1.7.0.xml new file mode 100644 index 00000000..6caee4d2 --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-1.7.0.xml @@ -0,0 +1,38 @@ + + + 1.7.0 + + + + ../../../core/target/accumulo-core-1.7.0.jar + ../../../minicluster/target/accumulo-minicluster-1.7.0.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.client.lexicoders.impl + org.apache.accumulo.core.client.mapreduce.lib.impl + org.apache.accumulo.core.data.thrift + org.apache.accumulo.minicluster.impl + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + diff --git a/test/compat/japi-compliance/japi-accumulo-master.xml b/test/compat/japi-compliance/japi-accumulo-master.xml new file mode 100644 index 00000000..2fc184e7 --- /dev/null +++ b/test/compat/japi-compliance/japi-accumulo-master.xml @@ -0,0 +1,38 @@ + + + 1.7.0-SNAPSHOT + + + + ../../../core/target/accumulo-core-1.7.0-SNAPSHOT.jar + ../../../minicluster/target/accumulo-minicluster-1.7.0-SNAPSHOT.jar + + + + org.apache.accumulo.core.client.impl + org.apache.accumulo.core.client.lexicoders.impl + org.apache.accumulo.core.client.mapreduce.lib.impl + org.apache.accumulo.core.data.thrift + org.apache.accumulo.minicluster.impl + + + + org.apache.accumulo.core.client + org.apache.accumulo.core.data + org.apache.accumulo.minicluster + diff --git a/test/merkle-replication/README b/test/merkle-replication/README new file mode 100644 index 00000000..b8924910 --- /dev/null +++ b/test/merkle-replication/README @@ -0,0 +1,65 @@ + + +Distributed Replication Test +=========================== + +Ingests random data into a table configured for replication, and then +verifies that the original table and the replicated table have equivalent +data using a Merkle tree. + +* Steps to run test + +1. Configure merkle-env.sh + + Be sure to set the username/password to connect to Accumulo and + the Accumulo instance name and ZooKeepers. The example defaults + to using the same instance as the source and destination. + + The randomness of the generated data can be controlled via some + parameters in this file. Note that if deletes are introduced, it + is very likely to cause an incorrect verification since the tombstone'ing + of that delete will differ on the source and remote due to the order + of minor compactions that occur. + +2. Run configure-replication.sh + + This script sets up the two instances for replication, creates + the tables that will be replicated and adds some splits to them. + + This is destructive to any existing replication configuration so + use it with care on a real instance. + +3. Run ingest-data.sh + + Ingests the configured amount of random data into the source + table. + +4. Run 'accumulo-cluster stop' && 'accumulo-cluster start' on the source instance + + A tabletserver in the source instance is likely to still be referencing + a WAL for a presently online tablet which will prevent that + file from being replicated. Stopping the local instance will ensure + that all WALs are candidate for replication. + +5. Run verify-data.sh + + This will compute the leaves merkle tree for the source and destination + tables and then compute the root hash for both. The root hash + is presented to the user. + + If the root hashes are equal, the test passed; otherwise, the test fails. diff --git a/test/merkle-replication/configure-replication.sh b/test/merkle-replication/configure-replication.sh new file mode 100755 index 00000000..44ebdd7e --- /dev/null +++ b/test/merkle-replication/configure-replication.sh @@ -0,0 +1,99 @@ +#!/usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +SOURCE="${BASH_SOURCE[0]}" +while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink + dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${dir}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +script=$( basename "${SOURCE}" ) +# Stop: Resolve Script Directory + +# Guess at ACCUMULO_HOME and ACCUMULO_CONF_DIR if not already defined +ACCUMULO_HOME=${ACCUMULO_HOME:-"${dir}/../../.."} +ACCUMULO_CONF_DIR=${ACCUMULO_CONF_DIR:-"$ACCUMULO_HOME/conf"} + +# Get the configuration values +. ./merkle-env.sh + +tmpdir=$(mktemp -dt "$0.XXXXXXXXXX") + +source_commands="${tmpdir}/source_commands.txt" + +echo 'Removing old tables and setting replication name on source' + +echo "deletetable -f $SOURCE_TABLE_NAME" >> $source_commands +echo "createtable $SOURCE_TABLE_NAME" >> $source_commands +echo "config -s replication.name=source" >> $source_commands +echo "quit" >> $source_commands + +# Source: drop and create tables, configure unique name for replication and grant perms +echo $SOURCE_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $SOURCE_ACCUMULO_USER -z \ + $SOURCE_INSTANCE $SOURCE_ZOOKEEPERS -f $source_commands + +destination_commands="${tmpdir}/destination_commands.txt" + +echo 'Removing old tables and setting replication name on destination' + +echo "deletetable -f $DESTINATION_TABLE_NAME" >> $destination_commands +echo "createtable $DESTINATION_TABLE_NAME" >> $destination_commands +echo "config -s replication.name=destination" >> $destination_commands +echo "quit" >> $destination_commands + +# Destination: drop and create tables, configure unique name for replication and grant perms +echo $DESTINATION_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $DESTINATION_ACCUMULO_USER -z \ + $DESTINATION_INSTANCE $DESTINATION_ZOOKEEPERS -f $destination_commands + +rm $source_commands +rm $destination_commands + +table_id=$(echo $DESTINATION_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $DESTINATION_ACCUMULO_USER -z \ + $DESTINATION_INSTANCE $DESTINATION_ZOOKEEPERS -e 'tables -l' | grep "${DESTINATION_TABLE_NAME}" \ + | grep -v "${DESTINATION_MERKLE_TABLE_NAME}" | awk '{print $3}') + +echo "Configuring $SOURCE_TABLE_NAME to replicate to $DESTINATION_TABLE_NAME (id=$table_id)" + +# Define our peer 'destination' with the ReplicaSystem impl, instance name and ZKs +echo "config -s replication.peer.destination=org.apache.accumulo.tserver.replication.AccumuloReplicaSystem,$DESTINATION_INSTANCE,$DESTINATION_ZOOKEEPERS" >> $source_commands +# Username for 'destination' +echo "config -s replication.peer.user.destination=$DESTINATION_ACCUMULO_USER" >> $source_commands +# Password for 'destination' +echo "config -s replication.peer.password.destination=$DESTINATION_ACCUMULO_PASSWORD" >> $source_commands +# Configure replication to 'destination' for $SOURCE_TABLE_NAME +echo "config -t $SOURCE_TABLE_NAME -s table.replication.target.destination=$table_id" >> $source_commands +# Enable replication for the table +echo "config -t $SOURCE_TABLE_NAME -s table.replication=true" >> $source_commands +echo "quit" >> $source_commands + +# Configure replication from source to destination and then enable it +echo $SOURCE_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $SOURCE_ACCUMULO_USER -z \ + $SOURCE_INSTANCE $SOURCE_ZOOKEEPERS -f $source_commands + +rm $source_commands + +# Add some splits to make ingest faster +echo 'Adding splits...' + +echo $SOURCE_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $SOURCE_ACCUMULO_USER -z \ + $SOURCE_INSTANCE $SOURCE_ZOOKEEPERS -e "addsplits -t $SOURCE_TABLE_NAME 1 2 3 4 5 6 7 8 9" + +echo $DESTINATION_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $DESTINATION_ACCUMULO_USER -z \ + $DESTINATION_INSTANCE $DESTINATION_ZOOKEEPERS -e "addsplits -t $DESTINATION_TABLE_NAME 1 2 3 4 5 6 7 8 9" + diff --git a/test/merkle-replication/ingest-data.sh b/test/merkle-replication/ingest-data.sh new file mode 100755 index 00000000..91b8cccf --- /dev/null +++ b/test/merkle-replication/ingest-data.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +SOURCE="${BASH_SOURCE[0]}" +while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink + dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${dir}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +script=$( basename "${SOURCE}" ) +# Stop: Resolve Script Directory + +# Guess at ACCUMULO_HOME and ACCUMULO_CONF_DIR if not already defined +ACCUMULO_HOME=${ACCUMULO_HOME:-"${dir}/../../.."} +ACCUMULO_CONF_DIR=${ACCUMULO_CONF_DIR:-"$ACCUMULO_HOME/conf"} + +# Get the configuration values +. ./merkle-env.sh + +# Ingest data into the source table +$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.replication.merkle.ingest.RandomWorkload --table $SOURCE_TABLE_NAME \ + -i $SOURCE_INSTANCE -z $SOURCE_ZOOKEEPERS -u $SOURCE_ACCUMULO_USER -p $SOURCE_ACCUMULO_PASSWORD -d $DELETE_PERCENT \ + -cf $MAX_CF -cq $MAX_CQ -r $MAX_ROW -n $NUM_RECORDS diff --git a/test/merkle-replication/merkle-env.sh b/test/merkle-replication/merkle-env.sh new file mode 100755 index 00000000..d4053943 --- /dev/null +++ b/test/merkle-replication/merkle-env.sh @@ -0,0 +1,48 @@ +# 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. + +# Random data will be written to this table +SOURCE_TABLE_NAME='replicationSource' +# Then replicated to this table +DESTINATION_TABLE_NAME='replicationDestination' + +# Merkle tree to be stored in this table for the source table +SOURCE_MERKLE_TABLE_NAME="${SOURCE_TABLE_NAME}_merkle" +# Merkle tree to be stored in this table for the destination table +DESTINATION_MERKLE_TABLE_NAME="${DESTINATION_TABLE_NAME}_merkle" + +# Connection information to Accumulo +SOURCE_ACCUMULO_USER="user" +SOURCE_ACCUMULO_PASSWORD="password" + +DESTINATION_ACCUMULO_USER="${SOURCE_ACCUMULO_USER}" +DESTINATION_ACCUMULO_PASSWORD="${SOURCE_ACCUMULO_PASSWORD}" + +SOURCE_INSTANCE="accumulo" +DESTINATION_INSTANCE="${SOURCE_INSTANCE}" + +SOURCE_ZOOKEEPERS="localhost" +DESTINATION_ZOOKEEPERS="${SOURCE_ZOOKEEPERS}" + +# Accumulo user to be configured on the destination instance +#REPLICATION_USER="${ACCUMULO_USER}" +#REPLICATION_PASSWORD="${ACCUMULO_PASSWORD}" + +# Control amount and distribution of data written +NUM_RECORDS=100000000 +MAX_ROW=1000000 +MAX_CF=10 +MAX_CQ=100 +DELETE_PERCENT=0 diff --git a/test/merkle-replication/verify-data.sh b/test/merkle-replication/verify-data.sh new file mode 100755 index 00000000..225d8927 --- /dev/null +++ b/test/merkle-replication/verify-data.sh @@ -0,0 +1,91 @@ +#!/usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +SOURCE="${BASH_SOURCE[0]}" +while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink + dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${dir}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +dir=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +script=$( basename "${SOURCE}" ) +# Stop: Resolve Script Directory + +# Guess at ACCUMULO_HOME and ACCUMULO_CONF_DIR if not already defined +ACCUMULO_HOME=${ACCUMULO_HOME:-"${dir}/../../.."} +ACCUMULO_CONF_DIR=${ACCUMULO_CONF_DIR:-"$ACCUMULO_HOME/conf"} + +# Get the configuration values +. ./merkle-env.sh + +tmpdir=$(mktemp -dt "$0.XXXXXXXXXX") + +splits=${tmpdir}/splits + +echo 1 >> $splits +echo 2 >> $splits +echo 3 >> $splits +echo 4 >> $splits +echo 5 >> $splits +echo 6 >> $splits +echo 7 >> $splits +echo 8 >> $splits +echo 9 >> $splits + +commands=${tmpdir}/commands + +# Generate leaves of merkle trees for source +echo "deletetable -f $SOURCE_MERKLE_TABLE_NAME" >> $commands +echo "createtable $SOURCE_MERKLE_TABLE_NAME" >> $commands +echo "quit" >> $commands + +echo $SOURCE_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $SOURCE_ACCUMULO_USER -z \ + $SOURCE_INSTANCE $SOURCE_ZOOKEEPERS -f $commands + +echo -e "\nGenerating merkle tree hashes for $SOURCE_TABLE_NAME" + +$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.replication.merkle.cli.GenerateHashes -t $SOURCE_TABLE_NAME \ + -o $SOURCE_MERKLE_TABLE_NAME -i $SOURCE_INSTANCE -z $SOURCE_ZOOKEEPERS -u $SOURCE_ACCUMULO_USER \ + -p $SOURCE_ACCUMULO_PASSWORD -nt 8 -hash MD5 --splits $splits + +rm $commands + +# Generate leaves of merkle trees for destination +echo "deletetable -f $DESTINATION_MERKLE_TABLE_NAME" >> $commands +echo "createtable $DESTINATION_MERKLE_TABLE_NAME" >> $commands +echo "quit" >> $commands + +echo $DESTINATION_ACCUMULO_PASSWORD | ${ACCUMULO_HOME}/bin/accumulo shell -u $DESTINATION_ACCUMULO_USER -z \ + $DESTINATION_INSTANCE $DESTINATION_ZOOKEEPERS -f $commands + +echo -e "\nGenerating merkle tree hashes for $DESTINATION_TABLE_NAME" + +$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.replication.merkle.cli.GenerateHashes -t $DESTINATION_TABLE_NAME \ + -o $DESTINATION_MERKLE_TABLE_NAME -i $DESTINATION_INSTANCE -z $DESTINATION_ZOOKEEPERS -u $DESTINATION_ACCUMULO_USER \ + -p $DESTINATION_ACCUMULO_PASSWORD -nt 8 -hash MD5 --splits $splits + +echo -e "\nComputing root hash:" + +#Compute root node of merkle tree +$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.replication.merkle.cli.ComputeRootHash -t $SOURCE_MERKLE_TABLE_NAME \ + -i $SOURCE_INSTANCE -z $SOURCE_ZOOKEEPERS -u $SOURCE_ACCUMULO_USER -p $SOURCE_ACCUMULO_PASSWORD -hash MD5 + +$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.replication.merkle.cli.ComputeRootHash -t $DESTINATION_MERKLE_TABLE_NAME \ + -i $DESTINATION_INSTANCE -z $DESTINATION_ZOOKEEPERS -u $DESTINATION_ACCUMULO_USER -p $DESTINATION_ACCUMULO_PASSWORD -hash MD5 + +rm -rf $tmpdir diff --git a/test/scalability/README.md b/test/scalability/README.md new file mode 100644 index 00000000..4f287d22 --- /dev/null +++ b/test/scalability/README.md @@ -0,0 +1,57 @@ + + +Apache Accumulo Scalability Tests +================================= + +The scalability test framework needs to be configured for your Accumulo +instance by performing the following steps. + +WARNING: Each scalability test rewrites your `conf/tservers` file and reinitializes +your Accumulo instance. Do not run these tests on a cluster holding essential +data. + +1. Make sure you have both `ACCUMULO_HOME` and `HADOOP_HOME` set in your + `$ACCUMULO_CONF_DIR/accumulo-env.sh.` + +2. Create a 'site.conf' file in the `conf` directory containing settings + needed by test nodes to connect to Accumulo, and to guide the tests. + + `$ cp conf/site.conf.example conf/site.conf` + +3. Create an 'Ingest.conf' file in the `conf` directory containing performance + settings for the Ingest test. (This test is currently the only scalability + test available.) + + `$ cp conf/Ingest.conf.example conf/Ingest.conf` + + Each test has a unique ID (e.g., "Ingest") which correlates with its test + code in: + + `org.apache.accumulo.test.scalability.tests.` + + This ID correlates with a config file: + + `conf/.conf` + +To run the test, specify its ID to the run.py script. + +> `$ nohup ./run.py Ingest > test1.log 2>&1 &` + +A timestamped directory will be created, and results are placed in it as each +test completes. + diff --git a/test/scalability/conf/Ingest.conf.example b/test/scalability/conf/Ingest.conf.example new file mode 100644 index 00000000..38800a17 --- /dev/null +++ b/test/scalability/conf/Ingest.conf.example @@ -0,0 +1,27 @@ +# 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. + +# Table to ingest into +TABLE=ingest +# Number of entries to ingest +NUM_ENTRIES=3000000 + +# Batch writer configuration +# Max memory (in bytes) that each ingester will use to buffer writes +MAX_MEMORY=100000000 +# Max time (in millis) that each ingester will buffer data +MAX_LATENCY=600000 +# Number of threads each ingester will use to write data +NUM_THREADS=4 diff --git a/test/scalability/conf/site.conf.example b/test/scalability/conf/site.conf.example new file mode 100644 index 00000000..e1ce08f5 --- /dev/null +++ b/test/scalability/conf/site.conf.example @@ -0,0 +1,27 @@ +# 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. + +# accumulo instance +INSTANCE_NAME=a12 +# zookeepers +ZOOKEEPERS=localhost +# username +USER=root +# password +PASSWORD=secret +# used to determine which nodes to run tests on +TSERVERS=/accumulo/conf/tservers +# sets the number of nodes for each test case +TEST_CASES=1,2,4,8,16 diff --git a/test/scalability/run.py b/test/scalability/run.py new file mode 100755 index 00000000..2b21a12b --- /dev/null +++ b/test/scalability/run.py @@ -0,0 +1,228 @@ +#!/usr/bin/python + +# 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. + + +import logging +import time +import os +import sys +from ConfigParser import ConfigParser +from subprocess import Popen, PIPE + +class JavaConfig: + '''Enable access to properities in java siteConfig file''' + def __init__(self, fname): + self.prop_d = {} + for line in open(fname): + line = line.strip(); + if line.startswith('#') or len(line) == 0: + continue + pair = line.split('=') + if len(pair) != 2: + log.error("Invalid property (%s)" % line) + continue + self.prop_d[pair[0].strip()] = pair[1].strip() + + def get(self, prop): + return self.prop_d[prop] + +def file_len(fname): + i=0 + for line in open(fname): + i += 1 + return i + +def runTest(testName, siteConfig, testDir, numNodes, fdata): + + log('Stopping accumulo') + syscall('$ACCUMULO_HOME/bin/accumulo-cluster stop') + + log('Creating tservers file for this test') + tserversPath = siteConfig.get('TSERVERS') + nodesPath = testDir+'/nodes/%d' % numNodes + syscall('head -n %d %s > %s' % (numNodes,tserversPath,nodesPath)) + + log('Copying tservers file to accumulo config') + syscall('cp '+nodesPath+' $ACCUMULO_CONF_DIR/tservers'); + + log('Removing /accumulo directory in HDFS') + syscall("hadoop fs -rmr /accumulo") + + log('Initializing new Accumulo instance') + instance = siteConfig.get('INSTANCE_NAME') + passwd = siteConfig.get('PASSWORD') + syscall('printf "%s\nY\n%s\n%s\n" | $ACCUMULO_HOME/bin/accumulo init' % (instance, passwd, passwd)) + + log('Starting new Accumulo instance') + syscall('$ACCUMULO_HOME/bin/accumulo-cluster start') + + sleepTime = 30 + if numNodes > 120: + sleepTime = int(numNodes / 4) + log('Sleeping for %d seconds' % sleepTime) + time.sleep(sleepTime) + + log('Setting up %s test' % testName) + syscall('$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.scalability.Run %s setup %s' % (testName, numNodes)) + + log('Sleeping for 5 seconds') + time.sleep(5) + + log('Starting %s clients' % testName) + numThreads = numNodes + if int(numNodes) > 128: + numThreads='128' + syscall('pssh -P -h %s -p %s "$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.scalability.Run %s client %s >/tmp/scale.out 2>/tmp/scale.err &" < /dev/null' % (nodesPath, numThreads, testName, numNodes)) + + log('Sleeping for 30 sec before checking how many clients started...') + time.sleep(30) + output = Popen(["hadoop fs -ls /accumulo-scale/clients"], stdout=PIPE, shell=True).communicate()[0] + num_clients = int(output.split()[1]) + log('%s clients started!' % num_clients) + + log('Waiting until %d clients finish.' % num_clients) + last = 0 + done = 0 + while done < num_clients: + time.sleep(5) + output = Popen(["hadoop fs -ls /accumulo-scale/results"], stdout=PIPE, shell=True).communicate()[0] + if not output: + sys.stdout.write('.') + sys.stdout.flush() + continue + done = int(output.split()[1]) + if done != last: + sys.stdout.write('.%s' % done) + else: + sys.stdout.write('.') + sys.stdout.flush() + last = done + sys.stdout.flush() + log('\nAll clients are finished!') + + log('Copying results from HDFS') + resultsDir = "%s/results/%s" % (testDir, numNodes) + syscall('hadoop fs -copyToLocal /accumulo-scale/results %s' % resultsDir) + + log('Calculating results from clients') + times = [] + totalMs = 0L + totalEntries = 0L + totalBytes = 0L + for fn in os.listdir(resultsDir): + for line in open('%s/%s' % (resultsDir,fn)): + words = line.split() + if words[0] == 'ELAPSEDMS': + ms = long(words[1].strip()) + totalMs += ms + times.append(ms) + totalEntries += long(words[2].strip()) + totalBytes += long(words[3].strip()) + times.sort() + + print times + numClients = len(times) + min = times[0] / 1000 + avg = (float(totalMs) / numClients) / 1000 + median = times[int(numClients/2)] / 1000 + max = times[numClients-1] / 1000 + q1 = times[int(numClients/4)] / 1000 + q3 = times[int((3*numClients)/4)] / 1000 + + log('Tservs\tClients\tMin\tQ1\tAvg\tMed\tQ3\tMax\tEntries\tMB') + resultStr = '%d\t%d\t%d\t%d\t%d\t%d\t%d\t%d\t%dM\t%d' % (numNodes, numClients, min, q1, avg, median, q3, max, totalEntries / 1000000, totalBytes / 1000000) + log(resultStr) + fdata.write(resultStr + '\n') + fdata.flush() + + time.sleep(5) + + log('Tearing down %s test' % testName) + syscall('$ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.scalability.Run %s teardown %s' % (testName, numNodes)) + + time.sleep(10) + +def syscall(cmd): + log('> %s' % cmd) + os.system(cmd) + +def run(cmd, **kwargs): + log.debug("Running %s", ' '.join(cmd)) + handle = Popen(cmd, stdout=PIPE, **kwargs) + out, err = handle.communicate() + log.debug("Result %d (%r, %r)", handle.returncode, out, err) + return handle.returncode + +def log(msg): + print msg + sys.stdout.flush() + +def main(): + + if not os.getenv('ACCUMULO_HOME'): + raise 'ACCUMULO_HOME needs to be set!' + + if not os.getenv('ACCUMULO_CONF_DIR'): + raise 'ACCUMULO_CONF_DIR needs to be set!' + + if not os.getenv('HADOOP_HOME'): + raise 'HADOOP_HOME needs to be set!' + + if len(sys.argv) != 2: + log('Usage: run.py ') + sys.exit() + testName = sys.argv[1] + + logging.basicConfig(level=logging.DEBUG) + + log('Creating test directory structure') + testDir = 'test-%d' % time.time() + nodesDir = testDir+'/nodes' + resultsDir = testDir+'/results' + syscall('mkdir %s' % testDir) + syscall('mkdir %s' % nodesDir) + syscall('mkdir %s' % resultsDir) + + log('Removing current /accumulo-scale directory') + syscall('hadoop fs -rmr /accumulo-scale') + + log('Creating new /accumulo-scale directory structure') + syscall('hadoop fs -mkdir /accumulo-scale') + syscall('hadoop fs -mkdir /accumulo-scale/clients') + syscall('hadoop fs -mkdir /accumulo-scale/results') + syscall('hadoop fs -chmod -R 777 /accumulo-scale') + + log('Copying config to HDFS') + syscall('hadoop fs -copyFromLocal ./conf /accumulo-scale/conf') + + siteConfig = JavaConfig('conf/site.conf'); + tserversPath = siteConfig.get('TSERVERS') + maxNodes = file_len(tserversPath) + + fdata = open('%s/scale.dat' % testDir, 'w') + fdata.write('Tservs\tClients\tMin\tAvg\tMed\tMax\tEntries\tMB\n') + + for numNodes in siteConfig.get('TEST_CASES').split(','): + log('Running %s test with %s nodes' % (testName, numNodes)) + if int(numNodes) > maxNodes: + logging.error('Skipping %r test case as tservers file %r contains only %r nodes', numNodes, tserversPath, maxNodes) + continue + runTest(testName, siteConfig, testDir, int(numNodes), fdata) + sys.stdout.flush() + +if __name__ == '__main__': + main() diff --git a/test/scale/agitator.txt b/test/scale/agitator.txt new file mode 100644 index 00000000..9715bf08 --- /dev/null +++ b/test/scale/agitator.txt @@ -0,0 +1,27 @@ +# 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. + +* run continuous ingest, continuous walkers, and agitator for 24 hours +* stop continuous ingest, walkers, and agitator +* run verification map reduce job + +This test should be run with the following configurations of the agitator. +* agitator does not kill master and only kills one tablet server at a time +* agitator does not kill master and only kills multiple tablet server at a time +* agitator kills master and only kills multiple tablet server at a time +* agitator kills master and does not kill tablet servers + +The agitator can also be run during verification. Another test is to lower +a tables split threshold during verification. diff --git a/test/scale/catastrophic.txt b/test/scale/catastrophic.txt new file mode 100644 index 00000000..322cab74 --- /dev/null +++ b/test/scale/catastrophic.txt @@ -0,0 +1,24 @@ +# 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. + +* run continuous ingest and continuous walkers for 24 hours +* stop continuous ingest and walkers +* disable idle compactions +* run verification map reduce job +* ensure there are entries in memory and then kill -9 all tablet servers and master +* restart accumulo +* allow log recoveries to run +* run verification map reduce job +* compare counts from two verification runs diff --git a/test/scale/deleteLargeTable.txt b/test/scale/deleteLargeTable.txt new file mode 100644 index 00000000..59dd6486 --- /dev/null +++ b/test/scale/deleteLargeTable.txt @@ -0,0 +1,16 @@ +# 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. + +Create and delete a large table (do this with accumulo.metadata split alot) see #2381 diff --git a/test/scale/restart.txt b/test/scale/restart.txt new file mode 100644 index 00000000..bab7cd91 --- /dev/null +++ b/test/scale/restart.txt @@ -0,0 +1,19 @@ +# 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. + + * create a continuous ingest table with 16k tablets + * start continuous ingest + * restart accumulo, but leave the ingesters running + * after accumulo is up, ingest should quickly resume to pre-restart rates diff --git a/test/stress/README.md b/test/stress/README.md new file mode 100644 index 00000000..9eb9fdcb --- /dev/null +++ b/test/stress/README.md @@ -0,0 +1,105 @@ + + +Accumulo Stress Test +==================== + +This is an attempt to observe the behavior Accumulo displays when compacting and +reading cells. There are two components to this package: +`org.apache.accumulo.test.stress.random.Write` and +`org.apache.accumulo.test.stress.random.Scan`. + +Since this test is meant to stress the system, there is no pre-defined test +period and no success criteria that we can programmatically check. During a +successful test period, the Accumulo cluster should remain stable and +functional. Possible failure conditions include component failures, such as +tablet servers running out of memory. + +The `org.apache.accumulo.test.stress.random.Write` utility provides facilities +for writing random sized cells. Users can configure minimum and maximum sized +portions of a cell. The portions users can configure are the row, column family, +column qualifier and value. Note that the sizes are uniformly distributed +between the minimum and maximum values. + +The `org.apache.accumulo.test.stress.random.Scan` utility provides users with +the ability to query tables generated by the Write. It will pick a tablet at +random and scan the entire range. The amount of times this process is performed +is user configurable. By default, it happens 1,024 times. Users can also specify +whether or not the scan should be isolated. + +There is no shared state intended by either of these services. This allows +multiple clients to be run in parallel, either on the same host or distributed +across hosts. + +## Prerequisites + +The utilities are run on a normal, initialized Accumulo cluster. By default, +they only work with a table named "stress_test". + +The start and stop scripts rely on pssh. Before running them, you will need +to install pssh on the machines that will be controlled by them. + +## Running + +Copy `stress-env.sh.example` to `stress-env.sh` and edit it, supplying at least +correct values for the USERPASS and INSTANCE variables. The remainder of this +file describes and provides defaults for many of the configuration options +for the stress utilities. + +### Individual Execution + +Before starting a fresh write test, clear out the test table using the Accumulo +shell. + + > deletetable -f stress_test + > createtable stress_test + +To run a writer: + + $ ./writer.sh + +The writer will begin logging into a "logs" subdirectory, using timestamped +filenames. You can stop the writer by killing it or using Control-C. + +To run a reader: + + $ ./reader.sh + +The reader logs like the writer. By default, the reader reads forever, but +it can be configured for a fixed number of scan iterations. + +### Group Execution + +To run writers across multiple machines, create a writers file and list each +host that will run a writer, one per line. + +Finally, to start and stop the writers: + + $ ./start-writers.sh + $ ./stop-writers.sh + +For readers, make a similar readers file, and use `start-readers.sh` and +`stop-readers.sh`. + +The start scripts copy out stress-env.sh to each of the writers and readers, +overwriting what is there, so you only need to edit the file on the host from +which you run those scripts. + +Note that in versions of Accumulo greater than 1.6.0, users should verify that +the cell sizes they supply meet the criteria set forth by the key constraints +on their tables. By default, tables are configured to reject keys that are +greater than 1MB. diff --git a/test/stress/reader.sh b/test/stress/reader.sh new file mode 100755 index 00000000..e2826321 --- /dev/null +++ b/test/stress/reader.sh @@ -0,0 +1,39 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +# Stop: Resolve Script Directory +LOG_DIR=${DIR}/logs +mkdir -p "$LOG_DIR" + +# Source environment +. "${DIR}/stress-env.sh" + +ts=$(date +%Y%m%d%H%M%S) +host=$(hostname) +# We want USERPASS to word split +"${ACCUMULO_HOME}/bin/accumulo org.apache.accumulo.test.stress.random.Scan" "$INSTANCE" $USERPASS "$SCAN_SEED" "$CONTINUOUS_SCAN" "$SCAN_BATCH_SIZE" \ + > "$LOG_DIR/${ts}_${host}_reader.out" \ + 2> "$LOG_DIR/${ts}_${host}_reader.err" diff --git a/test/stress/readers b/test/stress/readers new file mode 100644 index 00000000..80a4224a --- /dev/null +++ b/test/stress/readers @@ -0,0 +1,17 @@ +# 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. + +readerhost1 +readerhost2 diff --git a/test/stress/start-readers.sh b/test/stress/start-readers.sh new file mode 100755 index 00000000..6f651f49 --- /dev/null +++ b/test/stress/start-readers.sh @@ -0,0 +1,40 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR="$( cd -P "$( dirname "${SOURCE}" )" && pwd )" + SOURCE="$(readlink "${SOURCE}")" + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR="$( cd -P "$( dirname "${SOURCE}" )" && pwd )" +# Stop: Resolve Script Directory + +# Source environment +. "${DIR}/stress-env.sh" + +if [[ ! -f ${DIR}/readers ]]; then + echo readers file is missing + exit 1 +fi + +# Copy environment out +pscp -h "${DIR}/readers" "${DIR}/stress-env.sh" "${DIR}" + +pssh -h "${DIR}/readers" "nohup ${DIR}/reader.sh >${DIR}/reader.out 2>${DIR}/reader.err < /dev/null &" diff --git a/test/stress/start-writers.sh b/test/stress/start-writers.sh new file mode 100755 index 00000000..474117c4 --- /dev/null +++ b/test/stress/start-writers.sh @@ -0,0 +1,40 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +# Stop: Resolve Script Directory + +# Source environment +. "${DIR}/stress-env.sh" + +if [[ ! -f ${DIR}/writers ]]; then + echo writers file is missing + exit 1 +fi + +# Copy environment out +pscp -h "${DIR}/writers" "${DIR}/stress-env.sh" "${DIR}" + +pssh -h "${DIR}/writers" "nohup ${DIR}/writer.sh >${DIR}/writer.out 2>${DIR}/writer.err < /dev/null &" diff --git a/test/stress/stop-readers.sh b/test/stress/stop-readers.sh new file mode 100755 index 00000000..8bd4a6ff --- /dev/null +++ b/test/stress/stop-readers.sh @@ -0,0 +1,36 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +# Stop: Resolve Script Directory + +# Source environment +. "${DIR}/stress-env.sh" + +if [[ ! -f "${DIR}/readers" ]]; then + echo readers file is missing + exit 1 +fi +pssh -h "${DIR}/readers" "pkill -f '[o]rg.apache.accumulo.test.stress.random.Scan'" < /dev/null diff --git a/test/stress/stop-writers.sh b/test/stress/stop-writers.sh new file mode 100755 index 00000000..5c9e8d7c --- /dev/null +++ b/test/stress/stop-writers.sh @@ -0,0 +1,36 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +# Stop: Resolve Script Directory + +# Source environment +. "${DIR}/stress-env.sh" + +if [[ ! -f "${DIR}/writers" ]]; then + echo writers file is missing + exit 1 +fi +pssh -h "${DIR}/writers" "pkill -f '[o]rg.apache.accumulo.test.stress.random.Write'" < /dev/null diff --git a/test/stress/stress-env.sh.example b/test/stress/stress-env.sh.example new file mode 100644 index 00000000..6efb24f5 --- /dev/null +++ b/test/stress/stress-env.sh.example @@ -0,0 +1,60 @@ +# 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. + +ACCUMULO_HOME=${ACCUMULO_HOME:-/opt/accumulo} + +# Edit the credentials to match your system +USERPASS='-u root -p secret' +INSTANCE='-z localhost -i inst' + +### Read configuration ### + +# This is the seed for the range picking logic used by the scanner. +SCAN_SEED='--scan-seed 1337' + +# Controls the number of random tablets the scanner will read sequentially +#SCAN_ITERATIONS='--num-iterations 1024' + +# Alternatively, we can just continously scan +CONTINUOUS_SCAN='--continuous' + +# Controls whether or not the scan will be using an isolated scanner. Add this to the execution +#SCAN_ISOLATION='--isolate' + +# Sets the batch size for the scanner, use a lower number for large rows / cells +#SCAN_BATCH_SIZE='--scan-batch-size -1' + +### Write configuration ### + +# Edit these to change the range of each cell component. The size is in bytes. +ROW_RANGE='--min-row-size 128 --max-row-size 128' +CF_RANGE='--min-cf-size 128 --max-cf-size 128' +CQ_RANGE='--min-cq-size 128 --max-cq-size 128' +VALUE_RANGE='--min-value-size 1024 --max-value-size 2048' +ROW_WIDTH='--min-row-width 1 --max-row-width 10' + +# These are the seeds for the random number generates used to generate each cell component. +ROW_SEED='--row-seed 1' +CF_SEED='--cf-seed 2' +CQ_SEED='--cq-seed 3' +VALUE_SEED='--value-seed 4' +ROW_WIDTH_SEED='--row-width-seed 5' + +# This is the maximum number of cells to include in each mutation written out. +# A non-positive value implies no limit. +MAX_CELLS_PER_MUTATION='--max-cells-per-mutation -1' + +# This is the delay in milliseconds between writes. Use <= 0 for no delay. +WRITE_DELAY='--write-delay 0' diff --git a/test/stress/writer.sh b/test/stress/writer.sh new file mode 100755 index 00000000..71072420 --- /dev/null +++ b/test/stress/writer.sh @@ -0,0 +1,44 @@ +#! /usr/bin/env bash + +# 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. + +# Start: Resolve Script Directory +# Ref: http://stackoverflow.com/questions/59895/ +SOURCE="${BASH_SOURCE[0]}" +while [ -h "${SOURCE}" ]; do # resolve $SOURCE until the file is no longer a symlink + DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) + SOURCE=$(readlink "${SOURCE}") + [[ "${SOURCE}" != /* ]] && SOURCE="${DIR}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located +done +DIR=$( cd -P "$( dirname "${SOURCE}" )" && pwd ) +# Stop: Resolve Script Directory +LOG_DIR=${DIR}/logs +mkdir -p "$LOG_DIR" + +# Source environment +. "${DIR}/stress-env.sh" + +ts=$(date +%Y%m%d%H%M%S) +host=$(hostname) + +# TBD - --clear-table option + +# We want $USERPASS to word split +"${ACCUMULO_HOME}/bin/accumulo org.apache.accumulo.test.stress.random.Write" "$INSTANCE" $USERPASS "$ROW_RANGE" "$CF_RANGE" "$CQ_RANGE" "$VALUE_RANGE" \ + "$ROW_SEED" "$CF_SEED" "$CQ_SEED" "$VALUE_SEED" \ + "$ROW_WIDTH" "$ROW_WIDTH_SEED" "$MAX_CELLS_PER_MUTATION" "$WRITE_DELAY" \ + > "$LOG_DIR/${ts}_${host}_writer.out" \ + 2> "$LOG_DIR/${ts}_${host}_writer.err" diff --git a/test/stress/writers b/test/stress/writers new file mode 100644 index 00000000..0a065900 --- /dev/null +++ b/test/stress/writers @@ -0,0 +1,17 @@ +# 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. + +writerhost1 +writerhost2 diff --git a/test/test1/README.md b/test/test1/README.md new file mode 100644 index 00000000..00767b7d --- /dev/null +++ b/test/test1/README.md @@ -0,0 +1,46 @@ + + +Command to run from command line + +Can run this test with pre-existing splits; use the following command to create the table with +100 pre-existing splits + +> `$ ../../../bin/accumulo 'org.apache.accumulo.test.TestIngest' --createTable \ +-u root -p secret --splits 100 --rows 0` + +Could try running verify commands after stopping and restarting accumulo + +When write ahead log is implemented can try killing tablet server in middle of ingest + +Run 5 parallel ingesters and verify: + +> `$ . ingest_test.sh` +(wait) +`$ . verify_test.sh` +(wait) + +Overwrite previous ingest: +> `$ . ingest_test_2.sh` +(wait) +`$ . verify_test_2.sh` +(wait) + +Delete what was previously ingested: +> `$ . ingest_test_3.sh` +(wait) + diff --git a/test/test1/ingest_test.sh b/test/test1/ingest_test.sh new file mode 100755 index 00000000..65bf2bd9 --- /dev/null +++ b/test/test1/ingest_test.sh @@ -0,0 +1,22 @@ +#! /usr/bin/env bash + +# 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. + +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 1000000 --start 4000000 --cols 1 & diff --git a/test/test1/ingest_test_2.sh b/test/test1/ingest_test_2.sh new file mode 100755 index 00000000..c1c5f751 --- /dev/null +++ b/test/test1/ingest_test_2.sh @@ -0,0 +1,22 @@ +#! /usr/bin/env bash + +# 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. + +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 2 --size 50 --random 57 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 2 --size 50 --random 57 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 2 --size 50 --random 57 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 2 --size 50 --random 57 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 2 --size 50 --random 57 --rows 1000000 --start 4000000 --cols 1 & diff --git a/test/test1/ingest_test_3.sh b/test/test1/ingest_test_3.sh new file mode 100755 index 00000000..650e3fe4 --- /dev/null +++ b/test/test1/ingest_test_3.sh @@ -0,0 +1,22 @@ +#! /usr/bin/env bash + +# 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. + +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 3 --delete --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 3 --delete --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 3 --delete --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 3 --delete --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 3 --delete --rows 1000000 --start 4000000 --cols 1 & diff --git a/test/test1/verify_test.sh b/test/test1/verify_test.sh new file mode 100755 index 00000000..f91eb5ff --- /dev/null +++ b/test/test1/verify_test.sh @@ -0,0 +1,22 @@ +#! /usr/bin/env bash + +# 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. + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 1 --random 56 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 1 --random 56 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 1 --random 56 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 1 --random 56 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 1 --random 56 --rows 1000000 --start 4000000 --cols 1 & diff --git a/test/test1/verify_test_2.sh b/test/test1/verify_test_2.sh new file mode 100755 index 00000000..75e41193 --- /dev/null +++ b/test/test1/verify_test_2.sh @@ -0,0 +1,22 @@ +#! /usr/bin/env bash + +# 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. + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 2 --random 57 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 2 --random 57 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 2 --random 57 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 2 --random 57 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest -u root -p secret --size 50 --timestamp 2 --random 57 --rows 1000000 --start 4000000 --cols 1 & diff --git a/test/test2/README.md b/test/test2/README.md new file mode 100644 index 00000000..e60539c8 --- /dev/null +++ b/test/test2/README.md @@ -0,0 +1,27 @@ + + +Test Concurrent Read/Write +========================== + +Can run this test with pre-existing splits; use the following command to create the table with +100 pre-existing splits: + +> `$ ../../../bin/accumulo org.apache.accumulo.test.TestIngest --createTable \ +-u root -p secret --splits 100 --rows 0 +$ . concurrent.sh` + diff --git a/test/test2/concurrent.sh b/test/test2/concurrent.sh new file mode 100755 index 00000000..a759faf0 --- /dev/null +++ b/test/test2/concurrent.sh @@ -0,0 +1,99 @@ +#! /usr/bin/env bash + +# 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. + +USERPASS='-u root -p secret' +../../../bin/accumulo shell $USERPASS -e 'deletetable -f test_ingest' +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --rows 0 --createTable + +echo "ingesting first halves (0 to (500K - 1), 1M to (1.5M - 1), etc)" +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 4000000 --cols 1 & + +wait + +echo "ingesting second halves (500K to (1M - 1), 1.5M to (2M - 1), etc) and verifying first halves" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 5000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 5000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 5000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 5000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 5000000 --start 4000000 --cols 1 & + +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 1500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 2500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 3500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 5000000 --start 4500000 --cols 1 & + +wait + +echo "verifying complete range" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 4000000 --cols 1 & + +wait + +echo "ingesting first halves (0 to (500K - 1), 1M to (1.5M - 1), etc) w/ new timestamp AND verifying second half w/ old timestamp" + +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 4000000 --cols 1 & + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 500000 --start 500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 500000 --start 1500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 500000 --start 2500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 500000 --start 3500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 500000 --start 4500000 --cols 1 & + + +wait + +echo "ingesting second halves (500K to (1M - 1), 1.5M to (2M - 1), etc) and verifying first halves" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 500000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 500000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 500000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 500000 --start 4000000 --cols 1 & + +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 1500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 2500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 3500000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 2 --size 50 --random 57 --rows 500000 --start 4500000 --cols 1 & + +wait + +echo "verifying complete range" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 4000000 --cols 1 & + + +wait diff --git a/test/test3/README.md b/test/test3/README.md new file mode 100644 index 00000000..e63ae78a --- /dev/null +++ b/test/test3/README.md @@ -0,0 +1,22 @@ + + +Test Creating Big Non-Splittable Row +==================================== + +TBD + diff --git a/test/test3/bigrow.sh b/test/test3/bigrow.sh new file mode 100755 index 00000000..e508146e --- /dev/null +++ b/test/test3/bigrow.sh @@ -0,0 +1,27 @@ +#! /usr/bin/env bash + +# 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. + +USERPASS='-u root -p secret' +../../../bin/accumulo shell $USERPASS -e 'deletetable -f test_ingest' +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --rows 0 --createTable + +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 1 --start 5000000 --cols 2000000; +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 1000000 --start 0 --cols 1; +#../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 1 --start 5000000 --cols 2000000; +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 0 --cols 1; +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --timestamp 1 --size 50 --random 56 --rows 1000000 --start 7000000 --cols 1; +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 7000000 --cols 1; diff --git a/test/test4/README.md b/test/test4/README.md new file mode 100644 index 00000000..3be9182f --- /dev/null +++ b/test/test4/README.md @@ -0,0 +1,26 @@ + + +Test Bulk Importing Data +======================== + +Can run this test with pre-existing splits... use the following command to create the table with +100 pre-existing splits + +> `../../../bin/accumulo org.apache.accumulo.test.TestIngest --createTable \ +-u root -p secret --rows 0 --splits 100` + diff --git a/test/test4/bulk_import_test.sh b/test/test4/bulk_import_test.sh new file mode 100755 index 00000000..d43898af --- /dev/null +++ b/test/test4/bulk_import_test.sh @@ -0,0 +1,72 @@ +#! /usr/bin/env bash + +# 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. + +hadoop dfs -rmr /testmf +USERPASS='-u root -p secret' +../../../bin/accumulo shell $USERPASS -e 'deletetable -f test_ingest' +../../../bin/accumulo org.apache.accumulo.test.TestIngest $USERPASS --rows 0 --createTable --splits 100 + +echo "creating first set of map files" + +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf01 --timestamp 1 --size 50 --random 56 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf02 --timestamp 1 --size 50 --random 56 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf03 --timestamp 1 --size 50 --random 56 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf04 --timestamp 1 --size 50 --random 56 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf05 --timestamp 1 --size 50 --random 56 --rows 1000000 --start 4000000 --cols 1 & + +wait + +echo "bulk importing" + +hadoop dfs -rmr /testmfFail +../../../bin/accumulo org.apache.accumulo.test.BulkImportDirectory $USERPASS -t test_ingest -s /testmf -f /testmfFail + +echo "verifying" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 1 --random 56 --rows 1000000 --start 4000000 --cols 1 & + +wait + +hadoop dfs -rmr /testmf + +echo "creating second set of map files" + +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf01 --timestamp 2 --size 50 --random 57 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf02 --timestamp 2 --size 50 --random 57 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf03 --timestamp 2 --size 50 --random 57 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf04 --timestamp 2 --size 50 --random 57 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.TestIngest --rfile /testmf/mf05 --timestamp 2 --size 50 --random 57 --rows 1000000 --start 4000000 --cols 1 & + +wait + +echo "bulk importing" + +hadoop dfs -rmr /testmfFail +../../../bin/accumulo org.apache.accumulo.test.BulkImportDirectory $USERPASS -t test_ingest -s /testmf -f /testmfFail + +echo "creating second set of map files" + +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 0 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 1000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 2000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 3000000 --cols 1 & +../../../bin/accumulo org.apache.accumulo.test.VerifyIngest $USERPASS --size 50 --timestamp 2 --random 57 --rows 1000000 --start 4000000 --cols 1 & + diff --git a/test/upgrade/upgrade_test.sh b/test/upgrade/upgrade_test.sh new file mode 100755 index 00000000..a5c78633 --- /dev/null +++ b/test/upgrade/upgrade_test.sh @@ -0,0 +1,77 @@ +#! /usr/bin/env bash + +# 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. + + +if [[ $# != 1 ]] ; then + BASENAME=$(basename "$0") + echo "Usage: $BASENAME clean|dirty" + exit -1 +fi + +#this script test upgrade. This script is not self verifying, its output must be inspected for correctness. + +#set DIR to point to configured accumulo dirs. + +#TODO could support multinode configs, this script assumes single node config + +PREV=../../../accumulo-1.7.1 +CURR=../../ +DIR=/accumulo +BULK=/tmp/upt +INSTANCE=testUp + +pkill -f accumulo.start +hadoop fs -rmr "$DIR" +hadoop fs -rmr "$BULK" +hadoop fs -mkdir -p "$BULK/fail" + +"$PREV/bin/accumulo" init --clear-instance-name --instance-name $INSTANCE --password secret +"$PREV/bin/start-all.sh" + +"$PREV/bin/accumulo" org.apache.accumulo.test.TestIngest -i $INSTANCE -u root -p secret --timestamp 1 --size 50 --random 56 --rows 200000 --start 0 --cols 1 --createTable --splits 10 +"$PREV/bin/accumulo" org.apache.accumulo.test.TestIngest -i $INSTANCE -u root -p secret --rfile $BULK/bulk/test --timestamp 1 --size 50 --random 56 --rows 200000 --start 200000 --cols 1 + +echo -e "table test_ingest\nimportdirectory $BULK/bulk $BULK/fail false" | $PREV/bin/accumulo shell -u root -p secret +if [[ $1 == dirty ]]; then + pkill -9 -f accumulo.start +else + "$PREV/bin/stop-all.sh" +fi + +echo "==== Starting Current ===" + +"$CURR/bin/accumulo-cluster" start +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 400000 --start 0 --cols 1 -i $INSTANCE -u root -p secret +echo "compact -t test_ingest -w" | $CURR/bin/accumulo shell -u root -p secret +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 400000 --start 0 --cols 1 -i $INSTANCE -u root -p secret + + +"$CURR/bin/accumulo" org.apache.accumulo.test.TestIngest --timestamp 2 --size 50 --random 57 --rows 500000 --start 0 --cols 1 -i $INSTANCE -u root -p secret +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -i $INSTANCE -u root -p secret +echo "compact -t test_ingest -w" | $CURR/bin/accumulo shell -u root -p secret +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -i $INSTANCE -u root -p secret + +"$CURR/bin/accumulo-cluster" stop +"$CURR/bin/accumulo-cluster" start + +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -i $INSTANCE -u root -p secret + +pkill -9 -f accumulo.start +"$CURR/bin/accumulo-cluster" start + +"$CURR/bin/accumulo" org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -i $INSTANCE -u root -p secret +