ACCUMULO-1854 More unit tests and fix a bug.
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c5dc070f Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c5dc070f Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c5dc070f Branch: refs/heads/ACCUMULO-1854-multi-aif Commit: c5dc070f0c10c0f9b00647934edd35e22a6b036c Parents: 5d3c3d5 Author: Josh Elser <els...@apache.org> Authored: Wed Nov 6 14:46:05 2013 -0500 Committer: Josh Elser <els...@apache.org> Committed: Wed Nov 6 14:46:05 2013 -0500 ---------------------------------------------------------------------- .../core/client/mapreduce/InputFormatBase.java | 8 +- .../client/mapreduce/InputFormatBaseTest.java | 82 ++++++++++++++++++++ 2 files changed, 87 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/c5dc070f/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java ---------------------------------------------------------------------- diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java index 32240b7..7042f19 100644 --- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java +++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java @@ -229,9 +229,11 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> { // If we used the default sequence ID, add that into the list of configured sequences if (conf.getBoolean(DEFAULT_SEQ_USED, false)) { configuredSequences.add(DEFAULT_SEQUENCE); - for (String configuredSequence : configuredSequencesArray) { - configuredSequences.add(Integer.parseInt(configuredSequence)); - } + } + + // Add the rest of any sequences to our list + for (String configuredSequence : configuredSequencesArray) { + configuredSequences.add(Integer.parseInt(configuredSequence)); } int lastParsedSeqIndex = configuredSequences.size() - 1; http://git-wip-us.apache.org/repos/asf/accumulo/blob/c5dc070f/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputFormatBaseTest.java ---------------------------------------------------------------------- diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputFormatBaseTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputFormatBaseTest.java new file mode 100644 index 0000000..9d167a9 --- /dev/null +++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputFormatBaseTest.java @@ -0,0 +1,82 @@ +package org.apache.accumulo.core.client.mapreduce; + +import java.util.NoSuchElementException; + +import org.apache.accumulo.core.security.Authorizations; +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class InputFormatBaseTest { + + private Configuration conf; + + @Before + public void setup() { + conf = new Configuration(); + } + + @Test + public void testDefaultSequence() { + AccumuloInputFormat.setInputInfo(conf, "root", "password".getBytes(), "table", new Authorizations("foo")); + + Assert.assertEquals(0, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(-1, InputFormatBase.nextSequenceToProcess(conf)); + } + + @Test + public void testDefaultSequenceInputAndConnection() { + AccumuloInputFormat.setInputInfo(conf, "root", "password".getBytes(), "table", new Authorizations("foo")); + AccumuloInputFormat.setZooKeeperInstance(conf, "instance1", "zk1"); + + Assert.assertEquals(0, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(-1, InputFormatBase.nextSequenceToProcess(conf)); + } + + @Test + public void testDefaultWithCustomSequence() { + AccumuloInputFormat.setInputInfo(conf, "root", "password".getBytes(), "table", new Authorizations("foo")); + AccumuloInputFormat.setZooKeeperInstance(conf, "instance", "zk"); + + int seq = AccumuloInputFormat.nextSequence(conf); + + Assert.assertEquals(1, seq); + + AccumuloInputFormat.setInputInfo(conf, seq, "root1", "password1".getBytes(), "table1", new Authorizations("foo1")); + AccumuloInputFormat.setZooKeeperInstance(conf, seq, "instance1", "zk1"); + + Assert.assertEquals(0, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(1, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(-1, InputFormatBase.nextSequenceToProcess(conf)); + } + + @Test + public void testMultipleSequences() { + int seq = AccumuloInputFormat.nextSequence(conf); + + AccumuloInputFormat.setInputInfo(conf, seq, "root1", "password1".getBytes(), "table1", new Authorizations("foo1")); + AccumuloInputFormat.setZooKeeperInstance(conf, seq, "instance1", "zk1"); + + seq = AccumuloInputFormat.nextSequence(conf); + + AccumuloInputFormat.setInputInfo(conf, seq, "root2", "password2".getBytes(), "table2", new Authorizations("foo2")); + AccumuloInputFormat.setZooKeeperInstance(conf, seq, "instance2", "zk2"); + + seq = AccumuloInputFormat.nextSequence(conf); + + AccumuloInputFormat.setInputInfo(conf, seq, "root3", "password3".getBytes(), "table3", new Authorizations("foo3")); + AccumuloInputFormat.setZooKeeperInstance(conf, seq, "instance3", "zk3"); + + Assert.assertEquals(1, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(2, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(3, InputFormatBase.nextSequenceToProcess(conf)); + Assert.assertEquals(-1, InputFormatBase.nextSequenceToProcess(conf)); + } + + @Test(expected = NoSuchElementException.class) + public void testNoSequences() { + // When nothing was set, we should error + InputFormatBase.nextSequenceToProcess(conf); + } +}