Repository: accumulo
Updated Branches:
  refs/heads/master 4dfcb9dec -> a8577a1c5


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
deleted file mode 100644
index dd531c0..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-
-/**
- * AccumuloInputFormat which returns an "empty" RangeInputSplit
- */
-public class EmptySplitsAccumuloInputFormat extends AccumuloInputFormat {
-
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException {
-    List<InputSplit> oldSplits = super.getSplits(context);
-    List<InputSplit> newSplits = new ArrayList<InputSplit>(oldSplits.size());
-
-    // Copy only the necessary information
-    for (InputSplit oldSplit : oldSplits) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit newSplit = new 
org.apache.accumulo.core.client.mapreduce.RangeInputSplit(
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) 
oldSplit);
-      newSplits.add(newSplit);
-    }
-
-    return newSplits;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
deleted file mode 100644
index 7f5c7d8..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.core.client.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Test;
-
-public class InputTableConfigTest {
-  
-  private InputTableConfig tableQueryConfig;
-  
-  @Before
-  public void setUp() {
-    tableQueryConfig = new InputTableConfig();
-  }
-  
-  @Test
-  public void testSerialization_OnlyTable() throws IOException {
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-    
-    assertEquals(tableQueryConfig, actualConfig);
-  }
-  
-  @Test
-  public void testSerialization_ranges() throws IOException {
-    List<Range> ranges = new ArrayList<Range>();
-    ranges.add(new Range("a", "b"));
-    ranges.add(new Range("c", "d"));
-    tableQueryConfig.setRanges(ranges);
-    
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-    
-    assertEquals(ranges, actualConfig.getRanges());
-  }
-  
-  @Test
-  public void testSerialization_columns() throws IOException {
-    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-    columns.add(new Pair<Text,Text>(new Text("cf1"), new Text("cq1")));
-    columns.add(new Pair<Text,Text>(new Text("cf2"), null));
-    tableQueryConfig.fetchColumns(columns);
-    
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-    
-    assertEquals(actualConfig.getFetchedColumns(), columns);
-  }
-  
-  @Test
-  public void testSerialization_iterators() throws IOException {
-    List<IteratorSetting> settings = new ArrayList<IteratorSetting>();
-    settings.add(new IteratorSetting(50, "iter", "iterclass"));
-    settings.add(new IteratorSetting(55, "iter2", "iterclass2"));
-    tableQueryConfig.setIterators(settings);
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-    assertEquals(actualConfig.getIterators(), settings);
-    
-  }
-  
-  private byte[] serialize(InputTableConfig tableQueryConfig) throws 
IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    tableQueryConfig.write(new DataOutputStream(baos));
-    baos.close();
-    return baos.toByteArray();
-  }
-  
-  private InputTableConfig deserialize(byte[] bytes) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-    InputTableConfig actualConfig = new InputTableConfig(new 
DataInputStream(bais));
-    bais.close();
-    return actualConfig;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
deleted file mode 100644
index 80e8c28..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.core.client.mapreduce;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class RangeInputSplitTest {
-
-  @Test
-  public void testSimpleWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new 
Key("a"), new Key("b")), new String[]{"localhost"});
-    
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-    
-    RangeInputSplit newSplit = new RangeInputSplit();
-    
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-    
-    Assert.assertEquals(split.getTableName(), newSplit.getTableName());
-    Assert.assertEquals(split.getTableId(), newSplit.getTableId());
-    Assert.assertEquals(split.getRange(), newSplit.getRange());
-    Assert.assertTrue(Arrays.equals(split.getLocations(), 
newSplit.getLocations()));
-  }
-
-  @Test
-  public void testAllFieldsWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new 
Key("a"), new Key("b")), new String[]{"localhost"});
-    
-    Set<Pair<Text,Text>> fetchedColumns = new HashSet<Pair<Text,Text>>();
-    
-    fetchedColumns.add(new Pair<Text,Text>(new Text("colf1"), new 
Text("colq1")));
-    fetchedColumns.add(new Pair<Text,Text>(new Text("colf2"), new 
Text("colq2")));
-    
-    split.setAuths(new Authorizations("foo"));
-    split.setOffline(true);
-    split.setIsolatedScan(true);
-    split.setUsesLocalIterators(true);
-    split.setFetchedColumns(fetchedColumns);
-    split.setToken(new PasswordToken("password"));
-    split.setPrincipal("root");
-    split.setInstanceName("instance");
-    split.setMockInstance(true);
-    split.setZooKeepers("localhost");
-    split.setLogLevel(Level.WARN);
-    
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-    
-    RangeInputSplit newSplit = new RangeInputSplit();
-    
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-    
-    Assert.assertEquals(split.getRange(), newSplit.getRange());
-    Assert.assertArrayEquals(split.getLocations(), newSplit.getLocations());
-    
-    Assert.assertEquals(split.getAuths(), newSplit.getAuths());
-    Assert.assertEquals(split.isOffline(), newSplit.isOffline());
-    Assert.assertEquals(split.isIsolatedScan(), newSplit.isOffline());
-    Assert.assertEquals(split.usesLocalIterators(), 
newSplit.usesLocalIterators());
-    Assert.assertEquals(split.getFetchedColumns(), 
newSplit.getFetchedColumns());
-    Assert.assertEquals(split.getToken(), newSplit.getToken());
-    Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
-    Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
-    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
-    Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
-    Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
deleted file mode 100644
index fd207a1..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.core.client.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-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.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-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 org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * 
- */
-public class TokenFileTest {
-  private static AssertionError e1 = null;
-  private static final String PREFIX = TokenFileTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
-
-  private static class MRTokenFileTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) throws IOException, 
InterruptedException {
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, 
InterruptedException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        context.write(new Text(), m);
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + 
MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> 
<outputtable>");
-      }
-
-      String user = args[0];
-      String tokenFile = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      @SuppressWarnings("deprecation")
-      Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + 
System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = CachedConfiguration.getInstance();
-      conf.set("hadoop.tmp.dir", new File(args[1]).getParent());
-      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
-    }
-  }
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new 
File(System.getProperty("user.dir") + "/target"));
-
-  @Test
-  public void testMR() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new 
BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", 
i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    File tf = folder.newFile("root_test.pw");
-    PrintStream out = new PrintStream(tf);
-    String outString = new Credentials("root", new 
PasswordToken("")).serialize();
-    out.println(outString);
-    out.close();
-
-    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), 
TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-
-    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
-    Iterator<Entry<Key,Value>> iter = scanner.iterator();
-    assertTrue(iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-    assertFalse(iter.hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
deleted file mode 100644
index 1983470..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.core.client.mapreduce.lib.impl;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import 
org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.Test;
-
-/**
- * 
- */
-public class ConfiguratorBaseTest {
-
-  private static enum PrivateTestingEnum {
-    SOMETHING, SOMETHING_ELSE
-  }
-
-  @Test
-  public void testEnumToConfKey() {
-    assertEquals(this.getClass().getSimpleName() + 
".PrivateTestingEnum.Something",
-        ConfiguratorBase.enumToConfKey(this.getClass(), 
PrivateTestingEnum.SOMETHING));
-    assertEquals(this.getClass().getSimpleName() + 
".PrivateTestingEnum.SomethingElse",
-        ConfiguratorBase.enumToConfKey(this.getClass(), 
PrivateTestingEnum.SOMETHING_ELSE));
-  }
-
-  @Test
-  public void 
testSetConnectorInfoClassOfQConfigurationStringAuthenticationToken() throws 
AccumuloSecurityException {
-    Configuration conf = new Configuration();
-    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser", new 
PasswordToken("testPassword"));
-    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), 
conf));
-    AuthenticationToken token = 
ConfiguratorBase.getAuthenticationToken(this.getClass(), conf);
-    assertEquals(PasswordToken.class, token.getClass());
-    assertEquals(new PasswordToken("testPassword"), token);
-    assertEquals(
-        "inline:" + PasswordToken.class.getName() + ":" + 
Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(new 
PasswordToken("testPassword"))),
-        conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.ConnectorInfo.TOKEN)));
-  }
-
-  @Test
-  public void testSetConnectorInfoClassOfQConfigurationStringString() throws 
AccumuloSecurityException {
-    Configuration conf = new Configuration();
-    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser", 
"testFile");
-    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), 
conf));
-    assertEquals("file:testFile", 
conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.ConnectorInfo.TOKEN)));
-  }
-
-  @Test
-  public void testSetZooKeeperInstance() {
-    Configuration conf = new Configuration();
-    ConfiguratorBase.setZooKeeperInstance(this.getClass(), conf, new 
ClientConfiguration().withInstance("testInstanceName").withZkHosts("testZooKeepers")
-        .withSsl(true).withZkTimeout(1234));
-    ClientConfiguration clientConf = 
ClientConfiguration.deserialize(conf.get(ConfiguratorBase.enumToConfKey(this.getClass(),
-        ConfiguratorBase.InstanceOpts.CLIENT_CONFIG)));
-    assertEquals("testInstanceName", 
clientConf.get(ClientProperty.INSTANCE_NAME));
-    assertEquals("testZooKeepers", 
clientConf.get(ClientProperty.INSTANCE_ZK_HOST));
-    assertEquals("true", 
clientConf.get(ClientProperty.INSTANCE_RPC_SSL_ENABLED));
-    assertEquals("1234", clientConf.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
-    assertEquals(ZooKeeperInstance.class.getSimpleName(), 
conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.InstanceOpts.TYPE)));
-
-    Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf);
-    assertEquals(ZooKeeperInstance.class.getName(), 
instance.getClass().getName());
-    assertEquals("testInstanceName", ((ZooKeeperInstance) 
instance).getInstanceName());
-    assertEquals("testZooKeepers", ((ZooKeeperInstance) 
instance).getZooKeepers());
-    assertEquals(1234000, ((ZooKeeperInstance) 
instance).getZooKeepersSessionTimeOut());
-  }
-
-  @Test
-  public void testSetMockInstance() {
-    Configuration conf = new Configuration();
-    ConfiguratorBase.setMockInstance(this.getClass(), conf, 
"testInstanceName");
-    assertEquals("testInstanceName", 
conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.InstanceOpts.NAME)));
-    assertEquals(null, 
conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.InstanceOpts.ZOO_KEEPERS)));
-    assertEquals(MockInstance.class.getSimpleName(), 
conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), 
ConfiguratorBase.InstanceOpts.TYPE)));
-    Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf);
-    assertEquals(MockInstance.class.getName(), instance.getClass().getName());
-  }
-
-  @Test
-  public void testSetLogLevel() {
-    Configuration conf = new Configuration();
-    Level currentLevel = Logger.getLogger(this.getClass()).getLevel();
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.DEBUG);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.DEBUG, ConfiguratorBase.getLogLevel(this.getClass(), 
conf));
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.INFO);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.INFO, ConfiguratorBase.getLogLevel(this.getClass(), 
conf));
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.FATAL);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.FATAL, ConfiguratorBase.getLogLevel(this.getClass(), 
conf));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
----------------------------------------------------------------------
diff --git 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
 
b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
deleted file mode 100644
index 8fca169..0000000
--- 
a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.core.client.mapreduce.lib.partition;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.Test;
-
-public class RangePartitionerTest {
-  
-  private static Text[] cutArray = new Text[] {new Text("A"), new Text("B"), 
new Text("C")};
-  
-  @Test
-  public void testNoSubBins() throws IOException {
-    for (int i = -2; i < 2; ++i) {
-      checkExpectedBins(i, new String[] {"A", "B", "C"}, new int[] {0, 1, 2});
-      checkExpectedBins(i, new String[] {"C", "A", "B"}, new int[] {2, 0, 1});
-      checkExpectedBins(i, new String[] {"", "AA", "BB", "CC"}, new int[] {0, 
1, 2, 3});
-    }
-  }
-  
-  @Test
-  public void testSubBins() throws IOException {
-    checkExpectedRangeBins(2, new String[] {"A", "B", "C"}, new int[] {1, 3, 
5});
-    checkExpectedRangeBins(2, new String[] {"C", "A", "B"}, new int[] {5, 1, 
3});
-    checkExpectedRangeBins(2, new String[] {"", "AA", "BB", "CC"}, new int[] 
{1, 3, 5, 7});
-    
-    checkExpectedRangeBins(3, new String[] {"A", "B", "C"}, new int[] {2, 5, 
8});
-    checkExpectedRangeBins(3, new String[] {"C", "A", "B"}, new int[] {8, 2, 
5});
-    checkExpectedRangeBins(3, new String[] {"", "AA", "BB", "CC"}, new int[] 
{2, 5, 8, 11});
-    
-    checkExpectedRangeBins(10, new String[] {"A", "B", "C"}, new int[] {9, 19, 
29});
-    checkExpectedRangeBins(10, new String[] {"C", "A", "B"}, new int[] {29, 9, 
19});
-    checkExpectedRangeBins(10, new String[] {"", "AA", "BB", "CC"}, new int[] 
{9, 19, 29, 39});
-  }
-  
-  private RangePartitioner prepPartitioner(int numSubBins) throws IOException {
-    @SuppressWarnings("deprecation")
-    Job job = new Job();
-    RangePartitioner.setNumSubBins(job, numSubBins);
-    RangePartitioner rp = new RangePartitioner();
-    rp.setConf(job.getConfiguration());
-    return rp;
-  }
-  
-  private void checkExpectedRangeBins(int numSubBins, String[] strings, int[] 
rangeEnds) throws IOException {
-    assertTrue(strings.length == rangeEnds.length);
-    for (int i = 0; i < strings.length; ++i) {
-      int endRange = rangeEnds[i];
-      int startRange = endRange + 1 - numSubBins;
-      int part = prepPartitioner(numSubBins).findPartition(new 
Text(strings[i]), cutArray, numSubBins);
-      assertTrue(part >= startRange);
-      assertTrue(part <= endRange);
-    }
-  }
-  
-  private void checkExpectedBins(int numSubBins, String[] strings, int[] bins) 
throws IOException {
-    assertTrue(strings.length == bins.length);
-    for (int i = 0; i < strings.length; ++i) {
-      int bin = bins[i], part = prepPartitioner(numSubBins).findPartition(new 
Text(strings[i]), cutArray, numSubBins);
-      assertTrue(bin == part);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/resources/log4j.properties 
b/mapreduce/src/test/resources/log4j.properties
deleted file mode 100644
index 9f968f8..0000000
--- a/mapreduce/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,28 +0,0 @@
-# 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.
-
-log4j.rootLogger=INFO, CA
-log4j.appender.CA=org.apache.log4j.ConsoleAppender
-log4j.appender.CA.layout=org.apache.log4j.PatternLayout
-log4j.appender.CA.layout.ConversionPattern=[%t] %-5p %c %x - %m%n
-
-log4j.logger.org.apache.accumulo.core.iterators.system.VisibilityFilter=FATAL
-log4j.logger.org.apache.accumulo.core.iterators.user.TransformingIteratorTest$IllegalVisCompactionKeyTransformingIterator=FATAL
-log4j.logger.org.apache.accumulo.core.iterators.user.TransformingIteratorTest$IllegalVisKeyTransformingIterator=FATAL
-log4j.logger.org.apache.commons.vfs2.impl.DefaultFileSystemManager=WARN
-log4j.logger.org.apache.hadoop.mapred=ERROR
-log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter=ERROR
-log4j.logger.org.apache.hadoop.util.ProcessTree=ERROR
-log4j.logger.org.apache.accumulo.core.util.format=FATAL

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3a0953b..1529d14 100644
--- a/pom.xml
+++ b/pom.xml
@@ -75,7 +75,6 @@
     <module>trace</module>
     <module>core</module>
     <module>shell</module>
-    <module>mapreduce</module>
     <module>fate</module>
     <module>start</module>
     <module>examples/simple</module>
@@ -249,11 +248,6 @@
       </dependency>
       <dependency>
         <groupId>org.apache.accumulo</groupId>
-        <artifactId>accumulo-mapreduce</artifactId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.accumulo</groupId>
         <artifactId>accumulo-master</artifactId>
         <version>${project.version}</version>
       </dependency>

Reply via email to