You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by pu...@apache.org on 2016/07/21 12:50:26 UTC

[1/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Repository: incubator-rya
Updated Branches:
  refs/heads/develop c698b5603 -> 42895eac0


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/TestUtils.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/TestUtils.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/TestUtils.java
new file mode 100644
index 0000000..3bb35d9
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/TestUtils.java
@@ -0,0 +1,113 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.accumulo.core.client.Connector;
+import org.calrissian.mango.collect.CloseableIterable;
+import org.junit.Assert;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.accumulo.query.AccumuloRyaQueryEngine;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.api.persist.query.RyaQuery;
+
+public class TestUtils {
+    public static void verify(Connector connector, AccumuloRdfConfiguration conf, RyaStatement... ryaStatements)
+            throws RyaDAOException, IOException {
+        AccumuloRyaDAO dao = new AccumuloRyaDAO();
+        dao.setConnector(connector);
+        dao.setConf(conf);
+        dao.init();
+        AccumuloRyaQueryEngine engine = dao.getQueryEngine();
+        for (RyaStatement ryaStatement : ryaStatements) {
+            verify(ryaStatement, engine);
+        }
+        dao.destroy();
+    }
+
+    public static RyaStatement verify(RyaStatement ryaStatement, AccumuloRyaQueryEngine queryEngine)
+      throws RyaDAOException, IOException {
+        //check osp
+        CloseableIterable<RyaStatement> statements =
+          queryEngine.query(RyaQuery.builder(new RyaStatement(null, null, ryaStatement.getObject()))
+                                    .build());
+        try {
+            verifyFirstStatement(ryaStatement, statements);
+        } finally {
+            statements.close();
+        }
+
+        //check po
+        statements = queryEngine.query(RyaQuery.builder(
+          new RyaStatement(null, ryaStatement.getPredicate(),
+                           ryaStatement.getObject())).build());
+        try {
+            verifyFirstStatement(ryaStatement, statements);
+        } finally {
+            statements.close();
+        }
+
+        //check spo
+        RyaStatement result;
+        statements = queryEngine.query(RyaQuery.builder(
+          new RyaStatement(ryaStatement.getSubject(),
+                           ryaStatement.getPredicate(),
+                           ryaStatement.getObject())).build());
+        try {
+            result = verifyFirstStatement(ryaStatement, statements);
+        } finally {
+            statements.close();
+        }
+        return result;
+    }
+
+    private static RyaStatement verifyFirstStatement(
+      RyaStatement ryaStatement, CloseableIterable<RyaStatement> statements) {
+        final Iterator<RyaStatement> iterator = statements.iterator();
+        Assert.assertTrue(iterator.hasNext());
+        final RyaStatement first = iterator.next();
+        Assert.assertEquals(ryaStatement.getSubject(), first.getSubject());
+        Assert.assertEquals(ryaStatement.getPredicate(), first.getPredicate());
+        Assert.assertEquals(ryaStatement.getObject(), first.getObject());
+        Assert.assertEquals(ryaStatement.getContext(), first.getContext());
+        Assert.assertEquals(ryaStatement.getQualifer(), first.getQualifer());
+        // Test for equality if provided, otherwise test that these are empty
+        if (ryaStatement.getColumnVisibility() == null) {
+            Assert.assertEquals("Expected empty visibility.", 0, first.getColumnVisibility().length);
+        }
+        else {
+            Assert.assertArrayEquals("Mismatched visibilities.",
+                    ryaStatement.getColumnVisibility(), first.getColumnVisibility());
+        }
+        if (ryaStatement.getValue() == null) {
+            Assert.assertEquals("Expected empty value array.", 0, first.getValue().length);
+        }
+        else {
+            Assert.assertArrayEquals("Mismatched values.", ryaStatement.getValue(), first.getValue());
+        }
+        Assert.assertFalse(iterator.hasNext());
+        return first;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountToolTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountToolTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountToolTest.java
new file mode 100644
index 0000000..837457a
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountToolTest.java
@@ -0,0 +1,283 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.accumulo.mr.tools.AccumuloRdfCountTool;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.resolver.RdfToRyaConversions;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.mock.MockInstance;
+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.security.TablePermission;
+import org.apache.hadoop.io.Text;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Created by IntelliJ IDEA.
+ * Date: 4/24/12
+ * Time: 5:05 PM
+ * To change this template use File | Settings | File Templates.
+ */
+//@Ignore
+public class AccumuloRdfCountToolTest {
+
+    private String user = "user";
+    private String pwd = "pwd";
+    private String instance = AccumuloRdfCountToolTest.class.getSimpleName() + ".myinstance";
+    private String tablePrefix = "t_";
+    private Authorizations auths = Constants.NO_AUTHS;
+    private Connector connector;
+
+    private AccumuloRyaDAO dao;
+    private ValueFactory vf = new ValueFactoryImpl();
+    private AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+    static String litdupsNS = "urn:test:litdups#";
+
+    @Before
+    public void setUp() throws Exception {
+        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+        SecurityOperations secOps = connector.securityOperations();
+        secOps.createUser(user, pwd.getBytes(), auths);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
+
+        dao = new AccumuloRyaDAO();
+        dao.setConnector(connector);
+        conf.setTablePrefix(tablePrefix);
+        dao.setConf(conf);
+        dao.init();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        dao.destroy();
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+    }
+
+    @Test
+    public void testMR() throws Exception {
+        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
+        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10))));
+
+        AccumuloRdfCountTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Drdf.tablePrefix=" + tablePrefix,
+        });
+
+        Map<String, Key> expectedValues = new HashMap<String, Key>();
+        String row = test1.getData();
+        expectedValues.put(row,
+                new Key(new Text(row),
+                        RdfCloudTripleStoreConstants.SUBJECT_CF_TXT,
+                        RdfCloudTripleStoreConstants.EMPTY_TEXT));
+        row = pred1.getData();
+        expectedValues.put(row,
+                new Key(new Text(row),
+                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
+                        RdfCloudTripleStoreConstants.EMPTY_TEXT));
+        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
+        scanner.setRange(new Range());
+        int count = 0;
+        for (Map.Entry<Key, Value> entry : scanner) {
+            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
+            assertEquals(11, Long.parseLong(entry.getValue().toString()));
+            count++;
+        }
+        assertEquals(2, count);
+    }
+
+//    public void testMRObject() throws Exception {
+//        URI pred1 = vf.createURI(litdupsNS, "pred1");
+//        Literal literal = vf.createLiteral(0);
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test0"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test1"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test2"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test3"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test4"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test5"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test6"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test7"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test8"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test9"), pred1, literal));
+//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test10"), pred1, literal));
+//        dao.commit();
+//
+//        AccumuloRdfCountTool.main(new String[]{
+//                "-Dac.mock=true",
+//                "-Dac.instance=" + instance,
+//                "-Dac.username=" + user,
+//                "-Dac.pwd=" + pwd,
+//                "-Drdf.tablePrefix=" + tablePrefix,
+//        });
+//
+//        Map<String, Key> expectedValues = new HashMap<String, Key>();
+//        byte[] row_bytes = RdfCloudTripleStoreUtils.writeValue(literal);
+//        expectedValues.put(new String(row_bytes),
+//                new Key(new Text(row_bytes),
+//                        RdfCloudTripleStoreConstants.OBJ_CF_TXT,
+//                        RdfCloudTripleStoreConstants.INFO_TXT));
+//        row_bytes = RdfCloudTripleStoreUtils.writeValue(pred1);
+//        expectedValues.put(new String(row_bytes),
+//                new Key(new Text(row_bytes),
+//                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
+//                        RdfCloudTripleStoreConstants.INFO_TXT));
+//        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
+//        scanner.setRange(new Range());
+//        int count = 0;
+//        for (Map.Entry<Key, Value> entry : scanner) {
+//            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
+//            assertEquals(11, Long.parseLong(entry.getValue().toString()));
+//            count++;
+//        }
+//        assertEquals(2, count);
+//    }
+
+    @Test
+    public void testTTL() throws Exception {
+        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
+        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9))));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10))));
+
+        AccumuloRdfCountTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Dac.ttl=0",
+                "-Drdf.tablePrefix=" + tablePrefix,
+        });
+
+        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
+        scanner.setRange(new Range());
+        int count = 0;
+        for (Map.Entry<Key, Value> entry : scanner) {
+            count++;
+        }
+        assertEquals(0, count);
+    }
+
+    @Test
+    public void testContext() throws Exception {
+        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
+        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
+        RyaURI cntxt = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "cntxt"));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9)), cntxt));
+        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10)), cntxt));
+
+        AccumuloRdfCountTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Drdf.tablePrefix=" + tablePrefix,
+        });
+
+        Map<String, Key> expectedValues = new HashMap<String, Key>();
+        String row = test1.getData();
+        expectedValues.put(row,
+                new Key(new Text(row),
+                        RdfCloudTripleStoreConstants.SUBJECT_CF_TXT,
+                        new Text(cntxt.getData())));
+        row = pred1.getData();
+        expectedValues.put(row,
+                new Key(new Text(row),
+                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
+                        new Text(cntxt.getData())));
+        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
+        scanner.setRange(new Range());
+        int count = 0;
+        for (Map.Entry<Key, Value> entry : scanner) {
+            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
+            assertEquals(11, Long.parseLong(entry.getValue().toString()));
+            count++;
+        }
+        assertEquals(2, count);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/RdfFileInputToolTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/RdfFileInputToolTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/RdfFileInputToolTest.java
new file mode 100644
index 0000000..13c7d6d
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/RdfFileInputToolTest.java
@@ -0,0 +1,131 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.junit.Test;
+import org.openrdf.rio.RDFFormat;
+
+import junit.framework.TestCase;
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.mr.TestUtils;
+import mvm.rya.accumulo.mr.tools.RdfFileInputTool;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaType;
+import mvm.rya.api.domain.RyaURI;
+
+/**
+ * Created by IntelliJ IDEA.
+ * Date: 4/25/12
+ * Time: 10:51 AM
+ * To change this template use File | Settings | File Templates.
+ */
+public class RdfFileInputToolTest extends TestCase {
+
+    private String user = "user";
+    private String pwd = "pwd";
+    private String instance = RdfFileInputToolTest.class.getSimpleName() + ".myinstance";
+    private String tablePrefix = "t_";
+    private Authorizations auths = new Authorizations("test_auths");
+    private Connector connector;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        connector = new MockInstance(instance).getConnector(user, new PasswordToken(pwd));
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+        SecurityOperations secOps = connector.securityOperations();
+        secOps.createLocalUser(user, new PasswordToken(pwd));
+        secOps.changeUserAuthorizations(user, auths);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+    }
+
+    @Test
+    public void testNTriplesInput() throws Exception {
+        RdfFileInputTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Dac.auth=" + auths.toString(),
+                "-Dac.cv=" + auths.toString(),
+                "-Drdf.tablePrefix=" + tablePrefix,
+                "-Drdf.format=" + RDFFormat.NTRIPLES.getName(),
+                "src/test/resources/test.ntriples",
+        });
+        RyaStatement rs = new RyaStatement(new RyaURI("urn:lubm:rdfts#GraduateStudent01"),
+                new RyaURI("urn:lubm:rdfts#hasFriend"),
+                new RyaURI("urn:lubm:rdfts#GraduateStudent02"));
+        rs.setColumnVisibility(auths.toString().getBytes());
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix(tablePrefix);
+        conf.setAuths(auths.toString());
+        TestUtils.verify(connector, conf, rs);
+    }
+
+    @Test
+    public void testInputContext() throws Exception {
+        RdfFileInputTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Dac.auth=" + auths.toString(),
+                "-Dac.cv=" + auths.toString(),
+                "-Drdf.tablePrefix=" + tablePrefix,
+                "-Drdf.format=" + RDFFormat.TRIG.getName(),
+                "src/test/resources/namedgraphs.trig",
+        });
+        RyaStatement rs = new RyaStatement(new RyaURI("http://www.example.org/exampleDocument#Monica"),
+                new RyaURI("http://www.example.org/vocabulary#name"),
+                new RyaType("Monica Murphy"),
+                new RyaURI("http://www.example.org/exampleDocument#G1"));
+        rs.setColumnVisibility(auths.toString().getBytes());
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix(tablePrefix);
+        conf.setAuths(auths.toString());
+        TestUtils.verify(connector, conf, rs);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/Upgrade322ToolTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/Upgrade322ToolTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/Upgrade322ToolTest.java
new file mode 100644
index 0000000..a6b22e9
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/Upgrade322ToolTest.java
@@ -0,0 +1,275 @@
+package mvm.rya.accumulo.mr.tools;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.Constants;
+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.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.mock.MockInstance;
+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.core.security.TablePermission;
+import org.openrdf.model.vocabulary.XMLSchema;
+
+/*
+ * 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 junit.framework.TestCase;
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.accumulo.mr.TestUtils;
+import mvm.rya.accumulo.mr.tools.Upgrade322Tool;
+import mvm.rya.accumulo.query.AccumuloRyaQueryEngine;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaType;
+import mvm.rya.api.domain.RyaURI;
+
+/**
+ * Created by IntelliJ IDEA.
+ * Date: 4/25/12
+ * Time: 10:51 AM
+ * To change this template use File | Settings | File Templates.
+ */
+public class Upgrade322ToolTest extends TestCase {
+
+    private String user = "user";
+    private String pwd = "pwd";
+    private String instance = Upgrade322ToolTest.class.getSimpleName() + ".myinstance";
+    private String tablePrefix = "t_";
+    private Authorizations auths = Constants.NO_AUTHS;
+    private Connector connector;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+
+        final String spoTable = tablePrefix +
+                                RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX;
+        final String poTable = tablePrefix +
+                               RdfCloudTripleStoreConstants.TBL_PO_SUFFIX;
+        final String ospTable = tablePrefix +
+                                RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX;
+
+        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
+
+        connector.tableOperations().create(spoTable);
+        connector.tableOperations().create(poTable);
+        connector.tableOperations().create(ospTable);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+        SecurityOperations secOps = connector.securityOperations();
+        secOps.createUser(user, pwd.getBytes(), auths);
+        secOps.grantTablePermission(user, spoTable, TablePermission.READ);
+        secOps.grantTablePermission(user, poTable, TablePermission.READ);
+        secOps.grantTablePermission(user, ospTable, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
+
+        //load data
+        final BatchWriter ospWriter = connector
+          .createBatchWriter(ospTable, new BatchWriterConfig());
+        ospWriter.addMutation(getMutation("00000000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010/tracked-data-provenance/ns#longLit\u0001\u0004"));
+        ospWriter.addMutation(getMutation("00000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#intLit\u0001\u0005"));
+        ospWriter.addMutation(getMutation("00000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#byteLit\u0001\t"));
+        ospWriter.addMutation(getMutation("00001 1.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#doubleLit\u0001\u0006"));
+        ospWriter.addMutation(getMutation("10\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0000http" +
+        "://here/2010/tracked-data-provenance/ns#shortLit\u0001http://www.w3" +
+        ".org/2001/XMLSchema#short\u0001\b"));
+        ospWriter.addMutation(getMutation("10.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#floatLit\u0001http" +
+        "://www.w3.org/2001/XMLSchema#float\u0001\b"));
+        ospWriter.addMutation(getMutation("3.0.0\u0000urn:mvm.rya/2012/05#rts\u0000urn:mvm" +
+        ".rya/2012/05#version\u0001\u0003"));
+        ospWriter.addMutation(getMutation("9223370726404375807\u0000http://here/2010/tracked-data-provenance/ns" +
+        "#uuid10\u0000http://here/2010/tracked-data-provenance/ns#dateLit" +
+        "\u0001\u0007"));
+        ospWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#Created\u0000http://here" +
+        "/2010/tracked-data-provenance/ns#uuid10\u0000http://www.w3" +
+        ".org/1999/02/22-rdf-syntax-ns#type\u0001\u0002"));
+        ospWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#objectuuid1\u0000http" +
+        "://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010" +
+        "/tracked-data-provenance/ns#uriLit\u0001\u0002"));
+        ospWriter.addMutation(getMutation("stringLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#stringLit\u0001" +
+        "\u0003"));
+        ospWriter.addMutation(getMutation("true\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
+        "\u0000http://here/2010/tracked-data-provenance/ns#booleanLit\u0001\n"));
+        ospWriter.flush();
+        ospWriter.close();
+
+        final BatchWriter spoWriter = connector
+          .createBatchWriter(spoTable, new BatchWriterConfig());
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010/tracked-data-provenance/ns#longLit\u000000000000000000000010\u0001\u0004"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#intLit\u000000000000010\u0001\u0005"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#byteLit\u000000000010\u0001\t"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#doubleLit\u000000001 1.0\u0001\u0006"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10\u0000http" +
+                                          "://here/2010/tracked-data-provenance/ns#shortLit\u000010\u0001http://www.w3" +
+                                          ".org/2001/XMLSchema#short\u0001\b"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#floatLit\u0001http" +
+                                          "://www.w3.org/2001/XMLSchema#float\u000010.0\u0001\b"));
+        spoWriter.addMutation(getMutation("urn:mvm.rya/2012/05#rts\u0000urn:mvm" +
+                                          ".rya/2012/05#version\u00003.0.0\u0001\u0003"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns" +
+                                          "#uuid10\u0000http://here/2010/tracked-data-provenance/ns#dateLit" +
+                                          "\u00009223370726404375807\u0001\u0007"));
+        spoWriter.addMutation(getMutation("http://here" +
+                                          "/2010/tracked-data-provenance/ns#uuid10\u0000http://www.w3" +
+                                          ".org/1999/02/22-rdf-syntax-ns#type\u0000http://here/2010/tracked-data-provenance/ns#Created\u0001\u0002"));
+        spoWriter.addMutation(getMutation("http" +
+                                          "://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010" +
+                                          "/tracked-data-provenance/ns#uriLit\u0000http://here/2010/tracked-data-provenance/ns#objectuuid1\u0001\u0002"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#stringLit\u0000stringLit\u0001" +
+                                          "\u0003"));
+        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
+                                          "\u0000http://here/2010/tracked-data-provenance/ns#booleanLit\u0000true\u0001\n"));
+        spoWriter.flush();
+        spoWriter.close();
+
+        final BatchWriter poWriter = connector
+          .createBatchWriter(poTable, new BatchWriterConfig());
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#longLit\u000000000000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0004"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#intLit\u000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0005"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#byteLit\u000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\t"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#doubleLit\u000000001 1.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0006"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#shortLit\u000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001http://www.w3" +
+                                          ".org/2001/XMLSchema#short\u0001\b"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#floatLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001http" +
+                                          "://www.w3.org/2001/XMLSchema#float\u000010.0\u0001\b"));
+        poWriter.addMutation(getMutation("urn:mvm" +
+                                          ".rya/2012/05#version\u00003.0.0\u0000urn:mvm.rya/2012/05#rts\u0001\u0003"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#dateLit" +
+                                          "\u00009223370726404375807\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0007"));
+        poWriter.addMutation(getMutation("http://www.w3" +
+                                          ".org/1999/02/22-rdf-syntax-ns#type\u0000http://here/2010/tracked-data-provenance/ns#Created\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0002"));
+        poWriter.addMutation(getMutation("http://here/2010" +
+                                          "/tracked-data-provenance/ns#uriLit\u0000http://here/2010/tracked-data-provenance/ns#objectuuid1\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0002"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#stringLit\u0000stringLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001" +
+                                          "\u0003"));
+        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#booleanLit\u0000true\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\n"));
+        poWriter.flush();
+        poWriter.close();
+    }
+
+    public Mutation getMutation(String row) {
+        final Mutation mutation = new Mutation(row);
+        mutation.put("", "", "");
+        return mutation;
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().delete(
+          tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        connector.tableOperations().delete(
+          tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+    }
+
+    public void testUpgrade() throws Exception {
+        Upgrade322Tool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Drdf.tablePrefix=" + tablePrefix,
+        });
+
+        final AccumuloRdfConfiguration configuration = new AccumuloRdfConfiguration();
+        configuration.setTablePrefix(tablePrefix);
+        final AccumuloRyaDAO ryaDAO = new AccumuloRyaDAO();
+        ryaDAO.setConnector(connector);
+        ryaDAO.setConf(configuration);
+        ryaDAO.init();
+
+        final AccumuloRyaQueryEngine queryEngine = ryaDAO.getQueryEngine();
+
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#booleanLit"),
+          new RyaType(XMLSchema.BOOLEAN, "true")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#longLit"),
+          new RyaType(XMLSchema.LONG, "10")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#intLit"),
+          new RyaType(XMLSchema.INTEGER, "10")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#byteLit"),
+          new RyaType(XMLSchema.BYTE, "10")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#doubleLit"),
+          new RyaType(XMLSchema.DOUBLE, "10.0")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#dateLit"),
+          new RyaType(XMLSchema.DATETIME, "2011-07-12T06:00:00.000Z")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#stringLit"),
+          new RyaType("stringLit")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns#uriLit"),
+          new RyaURI("http://here/2010/tracked-data-provenance/ns" +
+                     "#objectuuid1")), queryEngine);
+        TestUtils.verify(new RyaStatement(
+          new RyaURI("urn:mvm.rya/2012/05#rts"),
+          new RyaURI("urn:mvm.rya/2012/05#version"),
+          new RyaType("3.0.0")), queryEngine);
+    }
+
+    public void printTableData(String tableName)
+      throws TableNotFoundException{
+        Scanner scanner = connector.createScanner(tableName, auths);
+        scanner.setRange(new Range());
+        for(Map.Entry<Key, Value> entry : scanner) {
+            final Key key = entry.getKey();
+            final Value value = entry.getValue();
+            System.out.println(key.getRow() + " " + key.getColumnFamily() + " " + key.getColumnQualifier() + " " + key.getTimestamp() + " " + value.toString());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/UpgradeObjectSerializationTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/UpgradeObjectSerializationTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/UpgradeObjectSerializationTest.java
new file mode 100644
index 0000000..e5fd293
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/tools/UpgradeObjectSerializationTest.java
@@ -0,0 +1,119 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 mvm.rya.api.resolver.impl.*;
+import org.junit.Test;
+
+import static mvm.rya.accumulo.mr.tools.Upgrade322Tool.UpgradeObjectSerialization;
+import static org.junit.Assert.*;
+
+public class UpgradeObjectSerializationTest {
+
+    @Test
+    public void testBooleanUpgrade() throws Exception {
+        String object = "true";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, BooleanRyaTypeResolver.BOOLEAN_LITERAL_MARKER);
+
+        assertEquals("1", upgrade);
+    }
+
+    @Test
+    public void testBooleanUpgradeFalse() throws Exception {
+        String object = "false";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, BooleanRyaTypeResolver.BOOLEAN_LITERAL_MARKER);
+
+        assertEquals("0", upgrade);
+    }
+
+    @Test
+    public void testByteUpgradeLowest() throws Exception {
+        String object = "-127";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, ByteRyaTypeResolver.LITERAL_MARKER);
+
+        assertEquals("81", upgrade);
+    }
+
+    @Test
+    public void testByteUpgradeHighest() throws Exception {
+        String object = "127";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, ByteRyaTypeResolver.LITERAL_MARKER);
+
+        assertEquals("7f", upgrade);
+    }
+
+    @Test
+    public void testLongUpgrade() throws Exception {
+        String object = "00000000000000000010";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, LongRyaTypeResolver.LONG_LITERAL_MARKER);
+
+        assertEquals("800000000000000a", upgrade);
+    }
+
+    @Test
+    public void testIntUpgrade() throws Exception {
+        String object = "00000000010";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, IntegerRyaTypeResolver.INTEGER_LITERAL_MARKER);
+
+        assertEquals("8000000a", upgrade);
+    }
+
+    @Test
+    public void testDateTimeUpgrade() throws Exception {
+        String object = "9223370726404375807";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, DateTimeRyaTypeResolver.DATETIME_LITERAL_MARKER);
+
+        assertEquals("800001311cee3b00", upgrade);
+    }
+
+    @Test
+    public void testDoubleUpgrade() throws Exception {
+        String object = "00001 1.0";
+        final UpgradeObjectSerialization upgradeObjectSerialization
+          = new UpgradeObjectSerialization();
+        final String upgrade = upgradeObjectSerialization
+          .upgrade(object, DoubleRyaTypeResolver.DOUBLE_LITERAL_MARKER);
+
+        assertEquals("c024000000000000", upgrade);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/resources/namedgraphs.trig
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/resources/namedgraphs.trig b/mapreduce/src/test/resources/namedgraphs.trig
new file mode 100644
index 0000000..b647632
--- /dev/null
+++ b/mapreduce/src/test/resources/namedgraphs.trig
@@ -0,0 +1,7 @@
+@prefix rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#> .
+@prefix xsd: <http://www.w3.org/2001/XMLSchema#> .
+@prefix swp: <http://www.w3.org/2004/03/trix/swp-1/> .
+@prefix dc: <http://purl.org/dc/elements/1.1/> .
+@prefix ex: <http://www.example.org/vocabulary#> .
+@prefix : <http://www.example.org/exampleDocument#> .
+:G1 { :Monica ex:name "Monica Murphy" . }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/resources/test.ntriples
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/resources/test.ntriples b/mapreduce/src/test/resources/test.ntriples
new file mode 100644
index 0000000..1ac8116
--- /dev/null
+++ b/mapreduce/src/test/resources/test.ntriples
@@ -0,0 +1,3 @@
+<urn:lubm:rdfts#GraduateStudent01> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent02> .
+<urn:lubm:rdfts#GraduateStudent02> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent03> .
+<urn:lubm:rdfts#GraduateStudent03> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent04> .

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 62238ff..ec4bf1c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -60,6 +60,7 @@ under the License.
         <module>common</module>
         <module>dao</module>
         <module>extras</module>
+        <module>mapreduce</module>
         <module>osgi</module>
         <module>pig</module>
         <module>sail</module>
@@ -149,6 +150,11 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.apache.rya</groupId>
+                <artifactId>rya.mapreduce</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rya</groupId>
                 <artifactId>rya.prospector</artifactId>
                 <version>${project.version}</version>
             </dependency>


[6/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Posted by pu...@apache.org.
Consolidated MapReduce API and applications into toplevel project.

Changes include:
- Made RdfFileInputFormat threaded, allowing it to handle all RDF input
- Added entity-centric indexing to RyaOutputFormat
- Added methods for using Rya input and output formats to AbstractAccumuloMRTool (renamed setupInputFormat to setupAccumuloInput since there are now multiple input options)
- Removed *NullIndexer classes, which were only used in RyaOutputFormat
- Removed StatementWritable, changed to RyaStatementWritable where applicable (for consistency)
- Fixed hashCode/compareTo/equals methods in RyaStatementWritable, RyaType, and RyaURI
- Minor renaming and repackaging (standalone tools go in "tools", "fileinput" and "utils" removed)
- Minor tweaks to AccumuloHDFSFileInputFormat, which seemed to be broken with Accumulo 1.6
- Documented code and added a MapReduce page to the manual
- Added "examples" package with one example
- Removed RyaStatementMapper and RyaStatementReducer: these simply insert records into Rya.
  This functionality can be achieved with RyaOutputFormat and the default mapper/reducer,
  so these two classes seem redundant.
- Removed BulkNtripsInputTool, BulkNtripsInputToolIndexing, RdfFileInputByLineTool,
  and RyaBatchWriterInputTool: Fixes to RdfFileInputFormat now allow RdfFileInputTool
  to now handle all the file input use cases (configurable secondary indexers, handles
  any format, scales), rendering the other file import tools redundant. (Previously, all
  five tools had largely overlapping but subtly different behavior.)


Project: http://git-wip-us.apache.org/repos/asf/incubator-rya/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-rya/commit/42895eac
Tree: http://git-wip-us.apache.org/repos/asf/incubator-rya/tree/42895eac
Diff: http://git-wip-us.apache.org/repos/asf/incubator-rya/diff/42895eac

Branch: refs/heads/develop
Commit: 42895eac0be3eabb836bd1cf0130cdd5951835c2
Parents: c698b56
Author: Jesse Hatfield <je...@parsons.com>
Authored: Thu Jun 30 15:50:08 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Thu Jul 21 08:45:17 2016 -0400

----------------------------------------------------------------------
 .../main/java/mvm/rya/api/domain/RyaType.java   |  38 +-
 .../java/mvm/rya/api/domain/RyaTypeTest.java    | 119 ++++
 dao/accumulo.rya/pom.xml                        |  69 ---
 .../rya/accumulo/mr/AbstractAccumuloMRTool.java | 164 -----
 .../accumulo/mr/RyaStatementInputFormat.java    |  94 ---
 .../mvm/rya/accumulo/mr/RyaStatementMapper.java |  85 ---
 .../rya/accumulo/mr/RyaStatementReducer.java    |  87 ---
 .../rya/accumulo/mr/RyaStatementWritable.java   | 150 -----
 .../accumulo/mr/eval/AccumuloRdfCountTool.java  | 258 --------
 .../mr/fileinput/BulkNtripsInputTool.java       | 369 ------------
 .../mr/fileinput/RdfFileInputByLineTool.java    | 251 --------
 .../mr/fileinput/RdfFileInputFormat.java        | 146 -----
 .../accumulo/mr/fileinput/RdfFileInputTool.java | 175 ------
 .../rya/accumulo/mr/upgrade/Upgrade322Tool.java | 240 --------
 .../mr/utils/AccumuloHDFSFileInputFormat.java   | 206 -------
 .../rya/accumulo/mr/utils/AccumuloProps.java    |  58 --
 .../java/mvm/rya/accumulo/mr/utils/MRUtils.java | 119 ----
 .../mvm/rya/accumulo/mr/RyaInputFormatTest.java | 225 -------
 .../mr/eval/AccumuloRdfCountToolTest.java       | 282 ---------
 .../mr/fileinput/RdfFileInputToolTest.java      | 146 -----
 .../accumulo/mr/upgrade/Upgrade322ToolTest.java | 319 ----------
 .../upgrade/UpgradeObjectSerializationTest.java | 119 ----
 .../src/test/resources/namedgraphs.trig         |   7 -
 .../src/test/resources/test.ntriples            |   1 -
 .../rya/accumulo/mr/NullFreeTextIndexer.java    | 102 ----
 .../mvm/rya/accumulo/mr/NullGeoIndexer.java     | 153 -----
 .../rya/accumulo/mr/NullTemporalIndexer.java    | 186 ------
 .../mvm/rya/accumulo/mr/RyaOutputFormat.java    | 329 ----------
 .../mvm/rya/accumulo/mr/StatementWritable.java  |  86 ---
 .../fileinput/BulkNtripsInputToolIndexing.java  | 227 -------
 .../mr/fileinput/RyaBatchWriterInputTool.java   | 243 --------
 extras/rya.manual/src/site/markdown/_index.md   |   1 +
 extras/rya.manual/src/site/markdown/index.md    |   1 +
 extras/rya.manual/src/site/markdown/loaddata.md |  15 +-
 .../rya.manual/src/site/markdown/mapreduce.md   | 107 ++++
 .../src/site/markdown/sm-firststeps.md          |   6 +-
 extras/rya.manual/src/site/site.xml             |   1 +
 extras/rya.reasoning/pom.xml                    |   4 +
 .../rya/reasoning/mr/AbstractReasoningTool.java |   4 +-
 .../mvm/rya/reasoning/mr/ConformanceTest.java   |   2 +-
 .../mvm/rya/reasoning/mr/MRReasoningUtils.java  |   2 +-
 extras/tinkerpop.rya/pom.xml                    |   4 +
 .../config/RyaGraphConfiguration.groovy         |   2 +-
 .../mvm/rya/blueprints/TstGremlinRya.groovy     |   2 +-
 .../config/RyaGraphConfigurationTest.groovy     |   2 +-
 .../sail/RyaSailVertexSequenceTest.groovy       |   2 +-
 mapreduce/pom.xml                               | 125 ++++
 .../rya/accumulo/mr/AbstractAccumuloMRTool.java | 305 ++++++++++
 .../mr/AccumuloHDFSFileInputFormat.java         | 161 +++++
 .../main/java/mvm/rya/accumulo/mr/MRUtils.java  | 317 ++++++++++
 .../mvm/rya/accumulo/mr/RdfFileInputFormat.java | 443 ++++++++++++++
 .../mvm/rya/accumulo/mr/RyaInputFormat.java     | 130 ++++
 .../mvm/rya/accumulo/mr/RyaOutputFormat.java    | 597 +++++++++++++++++++
 .../rya/accumulo/mr/RyaStatementWritable.java   | 256 ++++++++
 .../accumulo/mr/examples/TextOutputExample.java | 196 ++++++
 .../accumulo/mr/tools/AccumuloRdfCountTool.java | 258 ++++++++
 .../rya/accumulo/mr/tools/RdfFileInputTool.java |  91 +++
 .../rya/accumulo/mr/tools/Upgrade322Tool.java   | 241 ++++++++
 .../rya/accumulo/mr/RdfFileInputFormatTest.java | 180 ++++++
 .../mvm/rya/accumulo/mr/RyaInputFormatTest.java | 156 +++++
 .../rya/accumulo/mr/RyaOutputFormatTest.java    | 324 ++++++++++
 .../accumulo/mr/RyaStatementWritableTest.java   | 146 +++++
 .../java/mvm/rya/accumulo/mr/TestUtils.java     | 113 ++++
 .../mr/tools/AccumuloRdfCountToolTest.java      | 283 +++++++++
 .../accumulo/mr/tools/RdfFileInputToolTest.java | 131 ++++
 .../accumulo/mr/tools/Upgrade322ToolTest.java   | 275 +++++++++
 .../tools/UpgradeObjectSerializationTest.java   | 119 ++++
 mapreduce/src/test/resources/namedgraphs.trig   |   7 +
 mapreduce/src/test/resources/test.ntriples      |   3 +
 pom.xml                                         |   6 +
 70 files changed, 5152 insertions(+), 4919 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/common/rya.api/src/main/java/mvm/rya/api/domain/RyaType.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/mvm/rya/api/domain/RyaType.java b/common/rya.api/src/main/java/mvm/rya/api/domain/RyaType.java
index ab580d6..7a35253 100644
--- a/common/rya.api/src/main/java/mvm/rya/api/domain/RyaType.java
+++ b/common/rya.api/src/main/java/mvm/rya/api/domain/RyaType.java
@@ -80,19 +80,25 @@ public class RyaType implements Comparable {
         return sb.toString();
     }
 
+    /**
+     * Determine equality based on string representations of data and datatype.
+     * @param o The object to compare with
+     * @return true if the other object is also a RyaType and both data and datatype match.
+     */
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
+        if (o == null || !(o instanceof RyaType)) return false;
         RyaType ryaType = (RyaType) o;
-
         if (data != null ? !data.equals(ryaType.data) : ryaType.data != null) return false;
         if (dataType != null ? !dataType.equals(ryaType.dataType) : ryaType.dataType != null) return false;
-
         return true;
     }
 
+    /**
+     * Generate a hash based on the string representations of both data and datatype.
+     * @return A hash consistent with equals.
+     */
     @Override
     public int hashCode() {
         int result = dataType != null ? dataType.hashCode() : 0;
@@ -100,12 +106,30 @@ public class RyaType implements Comparable {
         return result;
     }
 
+    /**
+     * Define a natural ordering based on data and datatype.
+     * @param o The object to compare with
+     * @return 0 if both the data string and the datatype string representation match between the objects,
+     *          where matching is defined by string comparison or both being null;
+     *          Otherwise, an integer whose sign yields a consistent ordering.
+     */
     @Override
     public int compareTo(Object o) {
-        if (o != null && this.getClass().isInstance(o)) {
+        int result = -1;
+        if (o != null && o instanceof RyaType) {
+            result = 0;
             RyaType other = (RyaType) o;
-            return this.getData().compareTo(other.getData());
+            if (this.data != other.data) {
+                if (this.data == null) return 1;
+                if (other.data == null) return -1;
+                result = this.data.compareTo(other.data);
+            }
+            if (result == 0 && this.dataType != other.dataType) {
+                if (this.dataType == null) return 1;
+                if (other.dataType == null) return -1;
+                result = this.dataType.toString().compareTo(other.dataType.toString());
+            }
         }
-        return -1;
+        return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/common/rya.api/src/test/java/mvm/rya/api/domain/RyaTypeTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/mvm/rya/api/domain/RyaTypeTest.java b/common/rya.api/src/test/java/mvm/rya/api/domain/RyaTypeTest.java
new file mode 100644
index 0000000..6db6053
--- /dev/null
+++ b/common/rya.api/src/test/java/mvm/rya/api/domain/RyaTypeTest.java
@@ -0,0 +1,119 @@
+package mvm.rya.api.domain;
+
+/*
+ * 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 org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.vocabulary.XMLSchema;
+
+public class RyaTypeTest {
+    static RyaType a = new RyaType(XMLSchema.STRING, "http://www.example.com/Alice");
+    static RyaType b = new RyaType(XMLSchema.STRING, "http://www.example.com/Bob");
+    static RyaType c = new RyaType(XMLSchema.STRING, "http://www.example.com/Carol");
+    static RyaType aUri = new RyaType(XMLSchema.ANYURI, "http://www.example.com/Alice");
+    static RyaType bUri = new RyaType(XMLSchema.ANYURI, "http://www.example.com/Bob");
+    RyaType nullData = new RyaType(XMLSchema.STRING, null);
+    RyaType nullType = new RyaType(null, "http://www.example.com/Alice");
+    RyaType nullBoth = new RyaType(null, null);
+    RyaType same = new RyaType(XMLSchema.STRING, "http://www.example.com/Alice");
+
+    @Test
+    public void testCompareTo() throws Exception {
+        Assert.assertEquals("compareTo(self) should return zero.", 0, aUri.compareTo(aUri));
+        Assert.assertFalse("compareTo should return nonzero for different data and type.", aUri.compareTo(b) == 0);
+        Assert.assertFalse("compareTo should return nonzero for same data and different datatypes.", a.compareTo(aUri) == 0);
+        Assert.assertFalse("compareTo should return nonzero for same datatype and different data.", bUri.compareTo(aUri) == 0);
+        Assert.assertEquals("compareTo should return zero for different objects with matching data and datatype.",
+                0, a.compareTo(same));
+    }
+
+    @Test
+    public void testCompareToNullFields() throws Exception {
+        Assert.assertEquals("[has no nulls].compareTo([has null data]) should return -1", -1, a.compareTo(nullData));
+        Assert.assertEquals("[has no nulls].compareTo([has null type]) should return -1 if data is equal",
+                -1, a.compareTo(nullType));
+        Assert.assertEquals("[has null data].compareTo([has no nulls]) should return 1", 1, nullData.compareTo(a));
+        Assert.assertEquals("[has null type].compareTo([has no nulls]) should return 1 if data is equal",
+                 1, nullType.compareTo(a));
+        Assert.assertEquals("[has null type].compareTo([has null data]) should return -1", -1, nullType.compareTo(nullData));
+    }
+
+    @Test
+    public void testCompareToSymmetry() throws Exception {
+        int forward = Integer.signum(a.compareTo(b));
+        int backward = Integer.signum(b.compareTo(a));
+        Assert.assertEquals("Comparison of different values with same type should yield opposite signs.", forward, backward * -1);
+        forward = Integer.signum(bUri.compareTo(b));
+        backward = Integer.signum(b.compareTo(bUri));
+        Assert.assertEquals("Comparison of same values with different types should yield opposite signs.", forward, backward*-1);
+        forward = Integer.signum(aUri.compareTo(b));
+        backward = Integer.signum(b.compareTo(aUri));
+        Assert.assertEquals("Comparison of different values with different types should yield opposite signs.",
+                forward, backward * -1);
+    }
+
+    @Test
+    public void testCompareToTransitive() throws Exception {
+        int sign = Integer.signum(a.compareTo(b));
+        Assert.assertEquals("compareTo(a,b) and compareTo(b,c) should have the same sign.",
+                sign, Integer.signum(b.compareTo(c)));
+        Assert.assertEquals("if a > b > c, compareTo(a,c) should be consistent.", sign, Integer.signum(a.compareTo(c)));
+    }
+
+    @Test
+    public void testEquals() throws Exception {
+        Assert.assertTrue("Same data and datatype should be equal.", a.equals(same));
+        Assert.assertFalse("Same data, different datatype should be unequal.", a.equals(aUri));
+        Assert.assertFalse("Same datatype, different data should be unequal.", a.equals(b));
+    }
+
+    @Test
+    public void testEqualsNullFields() throws Exception {
+        Assert.assertFalse("equals(null) should return false.", a.equals(null));
+        Assert.assertFalse("Same data, one null datatype should be unequal.", a.equals(nullType));
+        Assert.assertFalse("Same datatype, one null data should be unequal.", a.equals(nullData));
+        RyaType sameNull = new RyaType(null, null);
+        Assert.assertTrue("Matching null fields should be equal.", sameNull.equals(nullBoth));
+    }
+
+    @Test
+    public void testEqualsCompareToConsistency() throws Exception {
+        Assert.assertEquals("equals and compareTo inconsistent for matching values and types.",
+                a.equals(same), a.compareTo(same) == 0);
+        Assert.assertEquals("equals and compareTo inconsistent for different values with same types.",
+                a.equals(b), a.compareTo(b) == 0);
+        Assert.assertEquals("equals and compareTo inconsistent for same values having different types.",
+                a.equals(aUri), a.compareTo(aUri) == 0);
+        Assert.assertEquals("equals and compareTo inconsistent for different values and different types.",
+                a.equals(bUri), a.compareTo(bUri) == 0);
+    }
+
+    @Test
+    public void testHashCodeEquals() throws Exception {
+        Assert.assertEquals("Same data and same type should yield same hash code.",
+                a.hashCode(), same.hashCode());
+        Assert.assertEquals("Same type and both null data should yield same hash code.",
+                nullData.hashCode(), new RyaType(XMLSchema.STRING, null).hashCode());
+        Assert.assertEquals("Same data and both null type should yield same hash code.",
+                nullType.hashCode(), new RyaType(null, "http://www.example.com/Alice").hashCode());
+        Assert.assertEquals("Null type and null data should yield same hash code.",
+                nullBoth.hashCode(), new RyaType(null, null).hashCode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/pom.xml
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/pom.xml b/dao/accumulo.rya/pom.xml
index c089cfe..295b755 100644
--- a/dao/accumulo.rya/pom.xml
+++ b/dao/accumulo.rya/pom.xml
@@ -36,84 +36,15 @@ under the License.
             <artifactId>rya.api</artifactId>
         </dependency>
         
-        <!-- Accumulo deps -->
         <dependency>
             <groupId>org.apache.accumulo</groupId>
             <artifactId>accumulo-core</artifactId>
         </dependency>
 
         <dependency>
-            <groupId>org.openrdf.sesame</groupId>
-            <artifactId>sesame-rio-ntriples</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.openrdf.sesame</groupId>
-            <artifactId>sesame-rio-nquads</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.openrdf.sesame</groupId>
-            <artifactId>sesame-queryalgebra-evaluation</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.openrdf.sesame</groupId>
-            <artifactId>sesame-rio-trig</artifactId>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.mrunit</groupId>
-            <artifactId>mrunit</artifactId>
-            <classifier>hadoop2</classifier>
-            <version>1.1.0</version>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
-
-    <build>
-        <pluginManagement>
-            <plugins>
-                <plugin>
-                    <groupId>org.apache.rat</groupId>
-                    <artifactId>apache-rat-plugin</artifactId>
-                    <configuration>
-                        <excludes>
-                            <!-- RDF data Files -->
-                            <exclude>**/*.ntriples</exclude>
-                            <exclude>**/*.trig</exclude>
-                        </excludes>
-                    </configuration>
-                </plugin>
-            </plugins>
-        </pluginManagement>
-    </build>
-
-    <profiles>
-        <profile>
-            <id>mr</id>
-            <build>
-                <plugins>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-shade-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <configuration>
-                                    <transformers>
-                                        <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                                    </transformers>
-                                </configuration>
-                            </execution>
-                        </executions>
-
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
-    </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
deleted file mode 100644
index 000c08a..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
+++ /dev/null
@@ -1,164 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 mvm.rya.accumulo.AccumuloRdfConstants;
-import mvm.rya.accumulo.mr.utils.AccumuloHDFSFileInputFormat;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.RdfCloudTripleStoreUtils;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.iterators.user.AgeOffFilter;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- */
-public abstract class AbstractAccumuloMRTool {
-
-    protected Configuration conf;
-    protected RdfCloudTripleStoreConstants.TABLE_LAYOUT rdfTableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP;
-    protected String userName = "root";
-    protected String pwd = "root";
-    protected String instance = "instance";
-    protected String zk = "zoo";
-    protected Authorizations authorizations = AccumuloRdfConstants.ALL_AUTHORIZATIONS;
-    protected String ttl = null;
-    protected boolean mock = false;
-    protected boolean hdfsInput = false;
-    protected String tablePrefix = RdfCloudTripleStoreConstants.TBL_PRFX_DEF;
-
-    protected void init() {
-        zk = conf.get(MRUtils.AC_ZK_PROP, zk);
-        ttl = conf.get(MRUtils.AC_TTL_PROP, ttl);
-        instance = conf.get(MRUtils.AC_INSTANCE_PROP, instance);
-        userName = conf.get(MRUtils.AC_USERNAME_PROP, userName);
-        pwd = conf.get(MRUtils.AC_PWD_PROP, pwd);
-        mock = conf.getBoolean(MRUtils.AC_MOCK_PROP, mock);
-        hdfsInput = conf.getBoolean(MRUtils.AC_HDFS_INPUT_PROP, hdfsInput);
-        tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, tablePrefix);
-        if (tablePrefix != null)
-            RdfCloudTripleStoreConstants.prefixTables(tablePrefix);
-        rdfTableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.valueOf(
-                conf.get(MRUtils.TABLE_LAYOUT_PROP, RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP.toString()));
-        String auth = conf.get(MRUtils.AC_AUTH_PROP);
-        if (auth != null)
-            authorizations = new Authorizations(auth.split(","));
-
-        if (!mock) {
-            conf.setBoolean("mapred.map.tasks.speculative.execution", false);
-            conf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
-            conf.set("io.sort.mb", "256");
-        }
-
-        //set ttl
-        ttl = conf.get(MRUtils.AC_TTL_PROP);
-    }
-
-    protected void setupInputFormat(Job job) throws AccumuloSecurityException {
-        // set up accumulo input
-        if (!hdfsInput) {
-            job.setInputFormatClass(AccumuloInputFormat.class);
-        } else {
-            job.setInputFormatClass(AccumuloHDFSFileInputFormat.class);
-        }
-        AccumuloInputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd));
-        AccumuloInputFormat.setInputTableName(job, RdfCloudTripleStoreUtils.layoutPrefixToTable(rdfTableLayout, tablePrefix));
-        AccumuloInputFormat.setScanAuthorizations(job, authorizations);
-        if (!mock) {
-            AccumuloInputFormat.setZooKeeperInstance(job, instance, zk);
-        } else {
-            AccumuloInputFormat.setMockInstance(job, instance);
-        }
-        if (ttl != null) {
-            IteratorSetting setting = new IteratorSetting(1, "fi", AgeOffFilter.class.getName());
-            AgeOffFilter.setTTL(setting, Long.valueOf(ttl));
-            AccumuloInputFormat.addIterator(job, setting);
-        }
-    }
-
-    protected void setupOutputFormat(Job job, String outputTable) throws AccumuloSecurityException {
-        AccumuloOutputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd));
-        AccumuloOutputFormat.setCreateTables(job, true);
-        AccumuloOutputFormat.setDefaultTableName(job, outputTable);
-        if (!mock) {
-            AccumuloOutputFormat.setZooKeeperInstance(job, instance, zk);
-        } else {
-            AccumuloOutputFormat.setMockInstance(job, instance);
-        }
-        job.setOutputFormatClass(AccumuloOutputFormat.class);
-    }
-
-    public void setConf(Configuration configuration) {
-        this.conf = configuration;
-    }
-
-    public Configuration getConf() {
-        return conf;
-    }
-
-    public String getInstance() {
-        return instance;
-    }
-
-    public void setInstance(String instance) {
-        this.instance = instance;
-    }
-
-    public String getPwd() {
-        return pwd;
-    }
-
-    public void setPwd(String pwd) {
-        this.pwd = pwd;
-    }
-
-    public String getZk() {
-        return zk;
-    }
-
-    public void setZk(String zk) {
-        this.zk = zk;
-    }
-
-    public String getTtl() {
-        return ttl;
-    }
-
-    public void setTtl(String ttl) {
-        this.ttl = ttl;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementInputFormat.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementInputFormat.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementInputFormat.java
deleted file mode 100644
index 3399166..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementInputFormat.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package mvm.rya.accumulo.mr;
-/*
- * 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 java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapreduce.AbstractInputFormat;
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RyaTripleContext;
-import mvm.rya.api.resolver.triple.TripleRow;
-import mvm.rya.api.resolver.triple.TripleRowResolverException;
-
-public class RyaStatementInputFormat extends AbstractInputFormat<Text, RyaStatementWritable> {
-    @Override
-    public RecordReader<Text, RyaStatementWritable> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
-        return new RyaStatementRecordReader();
-    }
-
-
-    public static void setTableLayout(Job conf, TABLE_LAYOUT layout) {
-        conf.getConfiguration().set(MRUtils.TABLE_LAYOUT_PROP, layout.name());
-    }
-
-    public class RyaStatementRecordReader extends AbstractRecordReader<Text, RyaStatementWritable> {
-
-        private RyaTripleContext ryaContext;
-        private TABLE_LAYOUT tableLayout;
-
-        @Override
-        protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName, RangeInputSplit split) {
-
-        }
-
-        @Override
-        public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-            super.initialize(inSplit, attempt);
-            this.tableLayout = TABLE_LAYOUT.valueOf(attempt.getConfiguration().get(MRUtils.TABLE_LAYOUT_PROP, TABLE_LAYOUT.OSP.toString()));
-            //TODO verify that this is correct
-            this.ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(attempt.getConfiguration()));
-        }
-
-        @Override
-        public boolean nextKeyValue() throws IOException {
-            if (!scannerIterator.hasNext())
-                return false;
-
-            Entry<Key, Value> entry = scannerIterator.next();
-            ++numKeysRead;
-            currentKey = entry.getKey();
-
-            try {
-                currentK = currentKey.getRow();
-                RyaStatement stmt = this.ryaContext.deserializeTriple(this.tableLayout, new TripleRow(entry.getKey().getRow().getBytes(), entry.getKey().getColumnFamily().getBytes(), entry.getKey().getColumnQualifier().getBytes(), entry.getKey().getTimestamp(), entry.getKey().getColumnVisibility().getBytes(), entry.getValue().get()));
-                RyaStatementWritable writable = new RyaStatementWritable();
-                writable.setRyaStatement(stmt);
-                currentV = writable;
-            } catch(TripleRowResolverException e) {
-                throw new IOException(e);
-            }
-            return true;
-        }
-
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementMapper.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementMapper.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementMapper.java
deleted file mode 100644
index d90215b..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementMapper.java
+++ /dev/null
@@ -1,85 +0,0 @@
-  
-package mvm.rya.accumulo.mr;
-/*
- * 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 static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_PO_SUFFIX;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_PRFX_DEF;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Mapper;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.RyaTableMutationsFactory;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.resolver.RyaTripleContext;
-
-public class RyaStatementMapper extends Mapper<Text, RyaStatementWritable, Text, Mutation> {
-
-    private Text spoTable;
-    private Text poTable;
-    private Text ospTable;
-    private RyaTableMutationsFactory mutationsFactory;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        String tablePrefix = context.getConfiguration().get(MRUtils.TABLE_PREFIX_PROPERTY, TBL_PRFX_DEF);
-        spoTable = new Text(tablePrefix + TBL_SPO_SUFFIX);
-        poTable = new Text(tablePrefix + TBL_PO_SUFFIX);
-        ospTable = new Text(tablePrefix + TBL_OSP_SUFFIX);
-
-        RyaTripleContext ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(context.getConfiguration()));
-        mutationsFactory = new RyaTableMutationsFactory(ryaContext);
-    }
-
-    @Override
-    protected void map(Text key, RyaStatementWritable value, Context context) throws IOException, InterruptedException {
-
-        Map<TABLE_LAYOUT, Collection<Mutation>> mutations = mutationsFactory.serialize(value.getRyaStatement());
-
-        for(TABLE_LAYOUT layout : mutations.keySet()) {
-
-            Text table = null;
-            switch (layout) {
-                case SPO:
-                    table = spoTable;
-                    break;
-                case OSP:
-                    table = ospTable;
-                    break;
-                case PO:
-                    table = poTable;
-                    break;
-            }
-
-            for(Mutation mutation : mutations.get(layout)) {
-                context.write(table, mutation);
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementReducer.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementReducer.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementReducer.java
deleted file mode 100644
index e353528..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementReducer.java
+++ /dev/null
@@ -1,87 +0,0 @@
-package mvm.rya.accumulo.mr;
-/*
- * 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 mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.RyaTableMutationsFactory;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.resolver.RyaTripleContext;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Reducer;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_PO_SUFFIX;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_PRFX_DEF;
-import static mvm.rya.api.RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX;
-
-public class RyaStatementReducer extends Reducer<WritableComparable, RyaStatementWritable, Text, Mutation> {
-
-    private Text spoTable;
-    private Text poTable;
-    private Text ospTable;
-    private RyaTableMutationsFactory mutationsFactory;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        String tablePrefix = context.getConfiguration().get(MRUtils.TABLE_PREFIX_PROPERTY, TBL_PRFX_DEF);
-        spoTable = new Text(tablePrefix + TBL_SPO_SUFFIX);
-        poTable = new Text(tablePrefix + TBL_PO_SUFFIX);
-        ospTable = new Text(tablePrefix + TBL_OSP_SUFFIX);
-
-        RyaTripleContext ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(context.getConfiguration()));
-        mutationsFactory = new RyaTableMutationsFactory(ryaContext);
-    }
-
-    @Override
-    protected void reduce(WritableComparable key, Iterable<RyaStatementWritable> values, Context context) throws IOException, InterruptedException {
-
-        for(RyaStatementWritable value : values) {
-            Map<TABLE_LAYOUT, Collection<Mutation>> mutations = mutationsFactory.serialize(value.getRyaStatement());
-
-            for(TABLE_LAYOUT layout : mutations.keySet()) {
-
-                Text table = null;
-                switch (layout) {
-                    case SPO:
-                        table = spoTable;
-                        break;
-                    case OSP:
-                        table = ospTable;
-                        break;
-                    case PO:
-                        table = poTable;
-                        break;
-                }
-
-                for(Mutation mutation : mutations.get(layout)) {
-                    context.write(table, mutation);
-                }
-            }
-
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
deleted file mode 100644
index 87a9433..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
+++ /dev/null
@@ -1,150 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.WritableComparable;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RyaTripleContext;
-import mvm.rya.api.resolver.triple.TripleRow;
-import mvm.rya.api.resolver.triple.TripleRowResolverException;
-
-/**
- * Date: 7/17/12
- * Time: 1:29 PM
- */
-public class RyaStatementWritable implements WritableComparable {
-
-    private RyaTripleContext ryaContext;
-    private RyaStatement ryaStatement;
-    
-    public RyaStatementWritable(Configuration conf) {
-        this();
-    }
-     
-    public RyaStatementWritable(RyaTripleContext ryaContext) {
-     	this.ryaContext = ryaContext;
-    }
-    
-    public RyaStatementWritable() {
-        this.ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration());
-    }
-    
-    public RyaStatementWritable(RyaStatement ryaStatement, RyaTripleContext ryaContext) {
-    	this(ryaContext);
-        this.ryaStatement = ryaStatement;
-    }
-
-    public RyaStatement getRyaStatement() {
-        return ryaStatement;
-    }
-
-    public void setRyaStatement(RyaStatement ryaStatement) {
-        this.ryaStatement = ryaStatement;
-    }
-
-    @Override
-    public int compareTo(Object o) {
-        if (o instanceof RyaStatementWritable) {
-            return (getRyaStatement().equals(((RyaStatementWritable) o).getRyaStatement())) ? (0) : (-1);
-        }
-        return -1;
-    }
-
-    @Override
-    public void write(DataOutput dataOutput) throws IOException {
-        if (ryaStatement == null) {
-            throw new IOException("Rya Statement is null");
-        }
-        try {
-            Map<RdfCloudTripleStoreConstants.TABLE_LAYOUT, TripleRow> map = ryaContext.serializeTriple(ryaStatement);
-            TripleRow tripleRow = map.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO);
-            byte[] row = tripleRow.getRow();
-            byte[] columnFamily = tripleRow.getColumnFamily();
-            byte[] columnQualifier = tripleRow.getColumnQualifier();
-            write(dataOutput, row);
-            write(dataOutput, columnFamily);
-            write(dataOutput, columnQualifier);
-            write(dataOutput, ryaStatement.getColumnVisibility());
-            write(dataOutput, ryaStatement.getValue());
-            Long timestamp = ryaStatement.getTimestamp();
-            boolean b = timestamp != null;
-            dataOutput.writeBoolean(b);
-            if (b) {
-                dataOutput.writeLong(timestamp);
-            }
-        } catch (TripleRowResolverException e) {
-            throw new IOException(e);
-        }
-    }
-
-    protected void write(DataOutput dataOutput, byte[] row) throws IOException {
-        boolean b = row != null;
-        dataOutput.writeBoolean(b);
-        if (b) {
-            dataOutput.writeInt(row.length);
-            dataOutput.write(row);
-        }
-    }
-
-    protected byte[] read(DataInput dataInput) throws IOException {
-        if (dataInput.readBoolean()) {
-            int len = dataInput.readInt();
-            byte[] bytes = new byte[len];
-            dataInput.readFully(bytes);
-            return bytes;
-        }else {
-            return null;
-        }
-    }
-
-    @Override
-    public void readFields(DataInput dataInput) throws IOException {
-        byte[] row = read(dataInput);
-        byte[] columnFamily = read(dataInput);
-        byte[] columnQualifier = read(dataInput);
-        byte[] columnVisibility = read(dataInput);
-        byte[] value = read(dataInput);
-        boolean b = dataInput.readBoolean();
-        Long timestamp = null;
-        if (b) {
-            timestamp = dataInput.readLong();
-        }
-        try {
-            ryaStatement = ryaContext.deserializeTriple(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO,
-                    new TripleRow(row, columnFamily, columnQualifier));
-            ryaStatement.setColumnVisibility(columnVisibility);
-            ryaStatement.setValue(value);
-            ryaStatement.setTimestamp(timestamp);
-        } catch (TripleRowResolverException e) {
-            throw new IOException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountTool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountTool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountTool.java
deleted file mode 100644
index ee1004d..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountTool.java
+++ /dev/null
@@ -1,258 +0,0 @@
-package mvm.rya.accumulo.mr.eval;
-
-/*
- * 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 java.io.IOException;
-import java.util.Date;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRdfConstants;
-import mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.resolver.RyaTripleContext;
-import mvm.rya.api.resolver.triple.TripleRow;
-import mvm.rya.api.resolver.triple.TripleRowResolverException;
-
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-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.ColumnVisibility;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import com.google.common.collect.Lists;
-import com.google.common.io.ByteArrayDataInput;
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
-
-/**
- * Count subject, predicate, object. Save in table
- * Class RdfCloudTripleStoreCountTool
- * Date: Apr 12, 2011
- * Time: 10:39:40 AM
- * @deprecated
- */
-public class AccumuloRdfCountTool extends AbstractAccumuloMRTool implements Tool {
-
-    public static void main(String[] args) {
-        try {
-
-            ToolRunner.run(new Configuration(), new AccumuloRdfCountTool(), args);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    /**
-     * cloudbase props
-     */
-
-    @Override
-    public int run(String[] strings) throws Exception {
-        conf.set(MRUtils.JOB_NAME_PROP, "Gather Evaluation Statistics");
-
-        //initialize
-        init();
-
-        Job job = new Job(conf);
-        job.setJarByClass(AccumuloRdfCountTool.class);
-        setupInputFormat(job);
-
-        AccumuloInputFormat.setRanges(job, Lists.newArrayList(new Range(new Text(new byte[]{}), new Text(new byte[]{Byte.MAX_VALUE}))));
-        // set input output of the particular job
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(Mutation.class);
-
-        // set mapper and reducer classes
-        job.setMapperClass(CountPiecesMapper.class);
-        job.setCombinerClass(CountPiecesCombiner.class);
-        job.setReducerClass(CountPiecesReducer.class);
-
-        String outputTable = tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX;
-        setupOutputFormat(job, outputTable);
-
-        // Submit the job
-        Date startTime = new Date();
-        System.out.println("Job started: " + startTime);
-        int exitCode = job.waitForCompletion(true) ? 0 : 1;
-
-        if (exitCode == 0) {
-            Date end_time = new Date();
-            System.out.println("Job ended: " + end_time);
-            System.out.println("The job took "
-                    + (end_time.getTime() - startTime.getTime()) / 1000
-                    + " seconds.");
-            return 0;
-        } else {
-            System.out.println("Job Failed!!!");
-        }
-
-        return -1;
-    }
-
-    public static class CountPiecesMapper extends Mapper<Key, Value, Text, LongWritable> {
-
-        public static final byte[] EMPTY_BYTES = new byte[0];
-        private RdfCloudTripleStoreConstants.TABLE_LAYOUT tableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP;
-
-        ValueFactoryImpl vf = new ValueFactoryImpl();
-
-        private Text keyOut = new Text();
-        private LongWritable valOut = new LongWritable(1);
-        private RyaTripleContext ryaContext;
-
-        @Override
-        protected void setup(Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-            tableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.valueOf(
-                    conf.get(MRUtils.TABLE_LAYOUT_PROP, RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP.toString()));
-            ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
-        }
-
-        @Override
-        protected void map(Key key, Value value, Context context) throws IOException, InterruptedException {
-            try {
-                RyaStatement statement = ryaContext.deserializeTriple(tableLayout, new TripleRow(key.getRow().getBytes(), key.getColumnFamily().getBytes(), key.getColumnQualifier().getBytes()));
-                //count each piece subject, pred, object
-
-                String subj = statement.getSubject().getData();
-                String pred = statement.getPredicate().getData();
-//                byte[] objBytes = tripleFormat.getValueFormat().serialize(statement.getObject());
-                RyaURI scontext = statement.getContext();
-                boolean includesContext = scontext != null;
-                String scontext_str = (includesContext) ? scontext.getData() : null;
-
-                ByteArrayDataOutput output = ByteStreams.newDataOutput();
-                output.writeUTF(subj);
-                output.writeUTF(RdfCloudTripleStoreConstants.SUBJECT_CF);
-                output.writeBoolean(includesContext);
-                if (includesContext)
-                    output.writeUTF(scontext_str);
-                keyOut.set(output.toByteArray());
-                context.write(keyOut, valOut);
-
-                output = ByteStreams.newDataOutput();
-                output.writeUTF(pred);
-                output.writeUTF(RdfCloudTripleStoreConstants.PRED_CF);
-                output.writeBoolean(includesContext);
-                if (includesContext)
-                    output.writeUTF(scontext_str);
-                keyOut.set(output.toByteArray());
-                context.write(keyOut, valOut);
-            } catch (TripleRowResolverException e) {
-                throw new IOException(e);
-            }
-        }
-    }
-
-    public static class CountPiecesCombiner extends Reducer<Text, LongWritable, Text, LongWritable> {
-
-        private LongWritable valOut = new LongWritable();
-
-        // TODO: can still add up to be large I guess
-        // any count lower than this does not need to be saved
-        public static final int TOO_LOW = 2;
-
-        @Override
-        protected void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
-            long count = 0;
-            for (LongWritable lw : values) {
-                count += lw.get();
-            }
-
-            if (count <= TOO_LOW)
-                return;
-
-            valOut.set(count);
-            context.write(key, valOut);
-        }
-
-    }
-
-    public static class CountPiecesReducer extends Reducer<Text, LongWritable, Text, Mutation> {
-
-        Text row = new Text();
-        Text cat_txt = new Text();
-        Value v_out = new Value();
-        ValueFactory vf = new ValueFactoryImpl();
-
-        // any count lower than this does not need to be saved
-        public static final int TOO_LOW = 10;
-        private String tablePrefix;
-        protected Text table;
-        private ColumnVisibility cv = AccumuloRdfConstants.EMPTY_CV;
-
-        @Override
-        protected void setup(Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            tablePrefix = context.getConfiguration().get(MRUtils.TABLE_PREFIX_PROPERTY, RdfCloudTripleStoreConstants.TBL_PRFX_DEF);
-            table = new Text(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-            final String cv_s = context.getConfiguration().get(MRUtils.AC_CV_PROP);
-            if (cv_s != null)
-                cv = new ColumnVisibility(cv_s);
-        }
-
-        @Override
-        protected void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
-            long count = 0;
-            for (LongWritable lw : values) {
-                count += lw.get();
-            }
-
-            if (count <= TOO_LOW)
-                return;
-
-            ByteArrayDataInput badi = ByteStreams.newDataInput(key.getBytes());
-            String v = badi.readUTF();
-            cat_txt.set(badi.readUTF());
-
-            Text columnQualifier = RdfCloudTripleStoreConstants.EMPTY_TEXT;
-            boolean includesContext = badi.readBoolean();
-            if (includesContext) {
-                columnQualifier = new Text(badi.readUTF());
-            }
-
-            row.set(v);
-            Mutation m = new Mutation(row);
-            v_out.set((count + "").getBytes());
-            m.put(cat_txt, columnQualifier, cv, v_out);
-            context.write(table, m);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputTool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputTool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputTool.java
deleted file mode 100644
index c3ddcfd..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputTool.java
+++ /dev/null
@@ -1,369 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 static com.google.common.base.Preconditions.checkNotNull;
-import static mvm.rya.accumulo.AccumuloRdfUtils.extractValue;
-import static mvm.rya.accumulo.AccumuloRdfUtils.from;
-
-import java.io.BufferedOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRdfConstants;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import mvm.rya.api.resolver.RyaTripleContext;
-import mvm.rya.api.resolver.triple.TripleRow;
-import mvm.rya.api.resolver.triple.TripleRowResolver;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat;
-import org.apache.accumulo.core.client.mapreduce.lib.partition.KeyRangePartitioner;
-import org.apache.accumulo.core.client.mapreduce.lib.partition.RangePartitioner;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.openrdf.model.Statement;
-import org.openrdf.rio.ParserConfig;
-import org.openrdf.rio.RDFFormat;
-import org.openrdf.rio.RDFHandler;
-import org.openrdf.rio.RDFHandlerException;
-import org.openrdf.rio.RDFParseException;
-import org.openrdf.rio.RDFParser;
-import org.openrdf.rio.nquads.NQuadsParser;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Take large ntrips files and use MapReduce and Cloudbase
- * Bulk ingest techniques to load into the table in our partition format.
- * <p/>
- * Input: NTrips file
- * Map:
- * - key : shard row - Text
- * - value : stmt in doc triple format - Text
- * Partitioner: RangePartitioner
- * Reduce:
- * - key : all the entries for each triple - Cloudbase Key
- * Class BulkNtripsInputTool
- * Date: Sep 13, 2011
- * Time: 10:00:17 AM
- */
-public class BulkNtripsInputTool extends Configured implements Tool {
-
-    public static final String WORKDIR_PROP = "bulk.n3.workdir";
-
-    private String userName = "root";
-    private String pwd = "root";
-    private String instance = "isntance";
-    private String zk = "zoo";
-    private String ttl = null;
-    private String workDirBase = "/temp/bulkcb/work";
-    private String format = RDFFormat.NQUADS.getName();
-
-    @Override
-    public int run(final String[] args) throws Exception {
-        final Configuration conf = getConf();
-        try {
-            //conf
-            zk = conf.get(MRUtils.AC_ZK_PROP, zk);
-            ttl = conf.get(MRUtils.AC_TTL_PROP, ttl);
-            instance = conf.get(MRUtils.AC_INSTANCE_PROP, instance);
-            userName = conf.get(MRUtils.AC_USERNAME_PROP, userName);
-            pwd = conf.get(MRUtils.AC_PWD_PROP, pwd);
-            workDirBase = conf.get(WORKDIR_PROP, workDirBase);
-            format = conf.get(MRUtils.FORMAT_PROP, format);
-            conf.set(MRUtils.FORMAT_PROP, format);
-            final String inputDir = args[0];
-
-            ZooKeeperInstance zooKeeperInstance = new ZooKeeperInstance(instance, zk);
-            Connector connector = zooKeeperInstance.getConnector(userName, new PasswordToken(pwd));
-            TableOperations tableOperations = connector.tableOperations();
-            
-            if (conf.get(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS) != null ) {
-                throw new IllegalArgumentException("Cannot use Bulk N Trips tool with Additional Indexers");
-            }
-
-            String tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, null);
-            if (tablePrefix != null)
-                RdfCloudTripleStoreConstants.prefixTables(tablePrefix);
-            String[] tables = {tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX,
-                    tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX,
-                    tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX};
-            Collection<Job> jobs = new ArrayList<Job>();
-            for (final String tableName : tables) {
-                PrintStream out = null;
-                try {
-                    String workDir = workDirBase + "/" + tableName;
-                    System.out.println("Loading data into table[" + tableName + "]");
-
-                    Job job = new Job(new Configuration(conf), "Bulk Ingest load data to Generic RDF Table[" + tableName + "]");
-                    job.setJarByClass(this.getClass());
-                    //setting long job
-                    Configuration jobConf = job.getConfiguration();
-                    jobConf.setBoolean("mapred.map.tasks.speculative.execution", false);
-                    jobConf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
-                    jobConf.set("io.sort.mb", jobConf.get("io.sort.mb", "256"));
-                    jobConf.setBoolean("mapred.compress.map.output", true);
-//                    jobConf.set("mapred.map.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec"); //TODO: I would like LZO compression
-
-                    job.setInputFormatClass(TextInputFormat.class);
-
-                    job.setMapperClass(ParseNtripsMapper.class);
-                    job.setMapOutputKeyClass(Key.class);
-                    job.setMapOutputValueClass(Value.class);
-
-                    job.setCombinerClass(OutStmtMutationsReducer.class);
-                    job.setReducerClass(OutStmtMutationsReducer.class);
-                    job.setOutputFormatClass(AccumuloFileOutputFormat.class);
-                   // AccumuloFileOutputFormat.setZooKeeperInstance(jobConf, instance, zk);
-
-                    jobConf.set(ParseNtripsMapper.TABLE_PROPERTY, tableName);
-
-                    TextInputFormat.setInputPaths(job, new Path(inputDir));
-
-                    FileSystem fs = FileSystem.get(conf);
-                    Path workPath = new Path(workDir);
-                    if (fs.exists(workPath))
-                        fs.delete(workPath, true);
-
-                    //make failures dir
-                    Path failures = new Path(workDir, "failures");
-                    fs.delete(failures, true);
-                    fs.mkdirs(new Path(workDir, "failures"));
-
-                    AccumuloFileOutputFormat.setOutputPath(job, new Path(workDir + "/files"));
-
-                    out = new PrintStream(new BufferedOutputStream(fs.create(new Path(workDir + "/splits.txt"))));
-
-                    if (!tableOperations.exists(tableName))
-                        tableOperations.create(tableName);
-                    Collection<Text> splits = tableOperations.getSplits(tableName, Integer.MAX_VALUE);
-                    for (Text split : splits)
-                        out.println(new String(Base64.encodeBase64(TextUtil.getBytes(split))));
-
-                    job.setNumReduceTasks(splits.size() + 1);
-                    out.close();
-
-                    job.setPartitionerClass(KeyRangePartitioner.class);
-                    RangePartitioner.setSplitFile(job, workDir + "/splits.txt");
-
-                    jobConf.set(WORKDIR_PROP, workDir);
-
-                    job.submit();
-                    jobs.add(job);
-
-                } catch (Exception re) {
-                    throw new RuntimeException(re);
-                } finally {
-                    if (out != null)
-                        out.close();
-                }
-            }
-
-            for (Job job : jobs) {
-                while (!job.isComplete()) {
-                    Thread.sleep(1000);
-                }
-            }
-
-            for(String tableName : tables) {
-                String workDir = workDirBase + "/" + tableName;
-                String filesDir = workDir + "/files";
-                String failuresDir = workDir + "/failures";
-                
-                FileSystem fs = FileSystem.get(conf);
-                
-                //make sure that the "accumulo" user can read/write/execute into these directories this path
-                fs.setPermission(new Path(filesDir), new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
-                fs.setPermission(new Path(failuresDir), new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
-                
-                tableOperations.importDirectory(
-                        tableName,
-                        filesDir,
-                        failuresDir,
-                        false);
-                
-            }
-
-        } catch (Exception e ){
-            throw new RuntimeException(e);
-        }
-
-        return 0;
-    }
-
-    public static void main(String[] args) throws Exception {
-    	ToolRunner.run(new Configuration(), new BulkNtripsInputTool(), args);
-    }
-
-    /**
-     * input: ntrips format triple
-     * <p/>
-     * output: key: shard row from generator
-     * value: stmt in serialized format (document format)
-     */
-    public static class ParseNtripsMapper extends Mapper<LongWritable, Text, Key, Value> {
-        public static final String TABLE_PROPERTY = "parsentripsmapper.table";
-
-        private RDFParser parser;
-        private String rdfFormat;
-        private String namedGraph;
-        private RyaTripleContext ryaContext;
-        private TripleRowResolver rowResolver;
-
-        @Override
-        protected void setup(final Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-            final String table = conf.get(TABLE_PROPERTY);
-            Preconditions.checkNotNull(table, "Set the " + TABLE_PROPERTY + " property on the map reduce job");
-            this.ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
-            rowResolver = ryaContext.getTripleResolver();
-
-            final String cv_s = conf.get(MRUtils.AC_CV_PROP);
-            final byte[] cv = cv_s == null ? null : cv_s.getBytes();
-            rdfFormat = conf.get(MRUtils.FORMAT_PROP);
-            checkNotNull(rdfFormat, "Rdf format cannot be null");
-
-            namedGraph = conf.get(MRUtils.NAMED_GRAPH_PROP);
-
-            parser = new NQuadsParser();
-    		parser.setParserConfig(new ParserConfig(true, true, true, RDFParser.DatatypeHandling.VERIFY));
-            parser.setRDFHandler(new RDFHandler() {
-
-                @Override
-                public void startRDF() throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void endRDF() throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void handleNamespace(String s, String s1) throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void handleStatement(Statement statement) throws RDFHandlerException {
-                    try {
-                        RyaStatement rs = RdfToRyaConversions.convertStatement(statement);
-                        if(rs.getColumnVisibility() == null) {
-                            rs.setColumnVisibility(cv);
-                        }
-
-                    	// Inject the specified context into the statement.
-                        if(namedGraph != null){
-                            rs.setContext(new RyaURI(namedGraph));
-                        } else if (statement.getContext() != null) {
-                            rs.setContext(new RyaURI(statement.getContext().toString()));
-                        } 
-
-                        Map<RdfCloudTripleStoreConstants.TABLE_LAYOUT,TripleRow> serialize = rowResolver.serialize(rs);
-
-                        if (table.contains(RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX)) {
-                            TripleRow tripleRow = serialize.get(TABLE_LAYOUT.SPO);
-                            context.write(
-                                    from(tripleRow),
-                                    extractValue(tripleRow)
-                            );
-                        } else if (table.contains(RdfCloudTripleStoreConstants.TBL_PO_SUFFIX)) {
-                            TripleRow tripleRow = serialize.get(TABLE_LAYOUT.PO);
-                            context.write(
-                                    from(tripleRow),
-                                    extractValue(tripleRow)
-                            );
-                        } else if (table.contains(RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX)) {
-                            TripleRow tripleRow = serialize.get(TABLE_LAYOUT.OSP);
-                            context.write(
-                                    from(tripleRow),
-                                    extractValue(tripleRow)
-                            );
-                        } else
-                            throw new IllegalArgumentException("Unrecognized table[" + table + "]");
-
-                    } catch (Exception e) {
-                        throw new RDFHandlerException(e);
-                    }
-                }
-
-                @Override
-                public void handleComment(String s) throws RDFHandlerException {
-
-                }
-            });
-        }
-
-        @Override
-        public void map(LongWritable key, Text value, Context output)
-                throws IOException, InterruptedException {
-            String rdf = value.toString();
-            try {
-                parser.parse(new StringReader(rdf), "");
-            } catch (RDFParseException e) {
-                System.out.println("Line[" + rdf + "] cannot be formatted with format[" + rdfFormat + "]. Exception[" + e.getMessage() + "]");
-            } catch (Exception e) {
-                e.printStackTrace();
-                throw new IOException("Exception occurred parsing triple[" + rdf + "]");
-            }
-        }
-    }
-
-    public static class OutStmtMutationsReducer extends Reducer<Key, Value, Key, Value> {
-
-        public void reduce(Key key, Iterable<Value> values, Context output)
-                throws IOException, InterruptedException {
-            output.write(key, AccumuloRdfConstants.EMPTY_VALUE);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputByLineTool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputByLineTool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputByLineTool.java
deleted file mode 100644
index 5a872a0..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputByLineTool.java
+++ /dev/null
@@ -1,251 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 java.io.IOException;
-import java.io.StringReader;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRdfConstants;
-import mvm.rya.accumulo.RyaTableMutationsFactory;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import mvm.rya.api.resolver.RyaTripleContext;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.openrdf.model.Statement;
-import org.openrdf.rio.RDFFormat;
-import org.openrdf.rio.RDFHandler;
-import org.openrdf.rio.RDFHandlerException;
-import org.openrdf.rio.RDFParser;
-import org.openrdf.rio.Rio;
-
-/**
- * Do bulk import of rdf files
- * Class RdfFileInputTool2
- * Date: May 16, 2011
- * Time: 3:12:16 PM
- */
-public class RdfFileInputByLineTool implements Tool {
-
-    private Configuration conf = new Configuration();
-
-    private String userName = "root";
-    private String pwd = "password";
-    private String instance = "instance";
-    private String zk = "zoo";
-    private String tablePrefix = null;
-    private RDFFormat format = RDFFormat.NTRIPLES;
-
-    public Configuration getConf() {
-        return conf;
-    }
-
-    public void setConf(Configuration conf) {
-        this.conf = conf;
-    }
-
-    public static void main(String[] args) {
-        try {
-            ToolRunner.run(new Configuration(), new RdfFileInputByLineTool(), args);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    public long runJob(String[] args) throws IOException, ClassNotFoundException, InterruptedException, AccumuloSecurityException {
-        conf.setBoolean("mapred.map.tasks.speculative.execution", false);
-        conf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
-        conf.set("io.sort.mb", "256");
-        conf.setLong("mapred.task.timeout", 600000000);
-
-        zk = conf.get(MRUtils.AC_ZK_PROP, zk);
-        instance = conf.get(MRUtils.AC_INSTANCE_PROP, instance);
-        userName = conf.get(MRUtils.AC_USERNAME_PROP, userName);
-        pwd = conf.get(MRUtils.AC_PWD_PROP, pwd);
-        format = RDFFormat.valueOf(conf.get(MRUtils.FORMAT_PROP, RDFFormat.NTRIPLES.toString()));
-
-        String tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, RdfCloudTripleStoreConstants.TBL_PRFX_DEF);
-
-        Job job = new Job(conf);
-        job.setJarByClass(RdfFileInputByLineTool.class);
-
-        // set up cloudbase input
-        job.setInputFormatClass(TextInputFormat.class);
-        FileInputFormat.addInputPath(job, new Path(args[0]));
-
-        // set input output of the particular job
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(Mutation.class);
-
-        job.setOutputFormatClass(AccumuloOutputFormat.class);
-        AccumuloOutputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd.getBytes()));
-        AccumuloOutputFormat.setCreateTables(job, true);
-        AccumuloOutputFormat.setDefaultTableName(job, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        AccumuloOutputFormat.setZooKeeperInstance(job, instance, zk);
-
-        // set mapper and reducer classes
-        job.setMapperClass(TextToMutationMapper.class);
-        job.setNumReduceTasks(0);
-
-        // Submit the job
-        Date startTime = new Date();
-        System.out.println("Job started: " + startTime);
-        int exitCode = job.waitForCompletion(true) ? 0 : 1;
-
-        if (exitCode == 0) {
-            Date end_time = new Date();
-            System.out.println("Job ended: " + end_time);
-            System.out.println("The job took "
-                    + (end_time.getTime() - startTime.getTime()) / 1000
-                    + " seconds.");
-            return job
-                    .getCounters()
-                    .findCounter("org.apache.hadoop.mapred.Task$Counter",
-                            "REDUCE_OUTPUT_RECORDS").getValue();
-        } else {
-            System.out.println("Job Failed!!!");
-        }
-
-        return -1;
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-        return (int) runJob(args);
-    }
-
-    public static class TextToMutationMapper extends Mapper<LongWritable, Text, Text, Mutation> {
-        protected RDFParser parser;
-        private String prefix;
-        private RDFFormat rdfFormat;
-        protected Text spo_table;
-        private Text po_table;
-        private Text osp_table;
-        private byte[] cv = AccumuloRdfConstants.EMPTY_CV.getExpression();
-
-        public TextToMutationMapper() {
-        }
-
-        @Override
-        protected void setup(final Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-            prefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, null);
-            if (prefix != null) {
-                RdfCloudTripleStoreConstants.prefixTables(prefix);
-            }
-
-            spo_table = new Text(prefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-            po_table = new Text(prefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-            osp_table = new Text(prefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-
-            final String cv_s = conf.get(MRUtils.AC_CV_PROP);
-            if (cv_s != null)
-                cv = cv_s.getBytes();
-
-            rdfFormat = RDFFormat.valueOf(conf.get(MRUtils.FORMAT_PROP, RDFFormat.NTRIPLES.toString()));
-            parser = Rio.createParser(rdfFormat);
-            RyaTripleContext tripleContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
-            final RyaTableMutationsFactory mut = new RyaTableMutationsFactory(tripleContext);
-
-            parser.setRDFHandler(new RDFHandler() {
-
-                @Override
-                public void startRDF() throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void endRDF() throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void handleNamespace(String s, String s1) throws RDFHandlerException {
-
-                }
-
-                @Override
-                public void handleStatement(Statement statement) throws RDFHandlerException {
-                    try {
-                        RyaStatement ryaStatement = RdfToRyaConversions.convertStatement(statement);
-                        if(ryaStatement.getColumnVisibility() == null) {
-                            ryaStatement.setColumnVisibility(cv);
-                        }
-                        Map<RdfCloudTripleStoreConstants.TABLE_LAYOUT, Collection<Mutation>> mutationMap =
-                                mut.serialize(ryaStatement);
-                        Collection<Mutation> spo = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO);
-                        Collection<Mutation> po = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.PO);
-                        Collection<Mutation> osp = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP);
-
-                        for (Mutation m : spo) {
-                            context.write(spo_table, m);
-                        }
-                        for (Mutation m : po) {
-                            context.write(po_table, m);
-                        }
-                        for (Mutation m : osp) {
-                            context.write(osp_table, m);
-                        }
-                    } catch (Exception e) {
-                        throw new RDFHandlerException(e);
-                    }
-                }
-
-                @Override
-                public void handleComment(String s) throws RDFHandlerException {
-
-                }
-            });
-        }
-
-        @Override
-        protected void map(LongWritable key, Text value, final Context context) throws IOException, InterruptedException {
-            try {
-                parser.parse(new StringReader(value.toString()), "");
-            } catch (Exception e) {
-                throw new IOException(e);
-            }
-        }
-
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputFormat.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputFormat.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputFormat.java
deleted file mode 100644
index 3d2fd78..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputFormat.java
+++ /dev/null
@@ -1,146 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 java.io.IOException;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.openrdf.model.Statement;
-import org.openrdf.rio.RDFFormat;
-import org.openrdf.rio.RDFHandler;
-import org.openrdf.rio.RDFHandlerException;
-import org.openrdf.rio.RDFParser;
-import org.openrdf.rio.Rio;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.mr.RyaStatementWritable;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import mvm.rya.api.resolver.RyaTripleContext;
-
-/**
- * Be able to input multiple rdf formatted files. Convert from rdf format to statements.
- * Class RdfFileInputFormat
- * Date: May 16, 2011
- * Time: 2:11:24 PM
- */
-public class RdfFileInputFormat extends FileInputFormat<LongWritable, RyaStatementWritable> {
-
-    @Override
-    public RecordReader<LongWritable, RyaStatementWritable> createRecordReader(InputSplit inputSplit,
-                                                                               TaskAttemptContext taskAttemptContext)
-            throws IOException, InterruptedException {
-        return new RdfFileRecordReader();
-    }
-
-    private class RdfFileRecordReader extends RecordReader<LongWritable, RyaStatementWritable> implements RDFHandler {
-
-        boolean closed = false;
-        long count = 0;
-        BlockingQueue<RyaStatementWritable> queue = new LinkedBlockingQueue<RyaStatementWritable>();
-        int total = 0;
-		private RyaTripleContext tripleContext;
-        
-
-        @Override
-        public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-            FileSplit fileSplit = (FileSplit) inputSplit;
-            Configuration conf = taskAttemptContext.getConfiguration();
-            String rdfForm_s = conf.get(MRUtils.FORMAT_PROP, RDFFormat.RDFXML.getName());
-            RDFFormat rdfFormat = RDFFormat.valueOf(rdfForm_s);
-            tripleContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
-
-            Path file = fileSplit.getPath();
-            FileSystem fs = file.getFileSystem(conf);
-            FSDataInputStream fileIn = fs.open(fileSplit.getPath());
-
-            RDFParser rdfParser = Rio.createParser(rdfFormat);
-            rdfParser.setRDFHandler(this);
-            try {
-                rdfParser.parse(fileIn, "");
-            } catch (Exception e) {
-                throw new IOException(e);
-            }
-            fileIn.close();
-            total = queue.size();
-            //TODO: Make this threaded so that you don't hold too many statements before sending them
-        }
-
-        @Override
-        public boolean nextKeyValue() throws IOException, InterruptedException {
-            return queue.size() > 0;
-        }
-
-        @Override
-        public LongWritable getCurrentKey() throws IOException, InterruptedException {
-            return new LongWritable(count++);
-        }
-
-        @Override
-        public RyaStatementWritable getCurrentValue() throws IOException, InterruptedException {
-            return queue.poll();
-        }
-
-        @Override
-        public float getProgress() throws IOException, InterruptedException {
-            return ((float) (total - queue.size())) / ((float) total);
-        }
-
-        @Override
-        public void close() throws IOException {
-            closed = true;
-        }
-
-        @Override
-        public void startRDF() throws RDFHandlerException {
-        }
-
-        @Override
-        public void endRDF() throws RDFHandlerException {
-        }
-
-        @Override
-        public void handleNamespace(String s, String s1) throws RDFHandlerException {
-        }
-
-        @Override
-        public void handleStatement(Statement statement) throws RDFHandlerException {
-            queue.add(new RyaStatementWritable(RdfToRyaConversions.convertStatement(statement), tripleContext));
-        }
-
-        @Override
-        public void handleComment(String s) throws RDFHandlerException {
-        }
-    }
-
-}



[5/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Posted by pu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputTool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputTool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputTool.java
deleted file mode 100644
index f44b6aa..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputTool.java
+++ /dev/null
@@ -1,175 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 static mvm.rya.accumulo.AccumuloRdfConstants.EMPTY_CV;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-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.openrdf.rio.RDFFormat;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.RyaTableMutationsFactory;
-import mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
-import mvm.rya.accumulo.mr.RyaStatementWritable;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RyaTripleContext;
-
-/**
- * Do bulk import of rdf files
- * Class RdfFileInputTool
- * Date: May 16, 2011
- * Time: 3:12:16 PM
- */
-public class RdfFileInputTool extends AbstractAccumuloMRTool implements Tool {
-
-    private String format = RDFFormat.RDFXML.getName();
-
-    public static void main(String[] args) {
-        try {
-            ToolRunner.run(new Configuration(), new RdfFileInputTool(), args);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    public long runJob(String[] args) throws IOException, ClassNotFoundException, InterruptedException, AccumuloSecurityException {
-        conf.set(MRUtils.JOB_NAME_PROP, "Rdf File Input");
-        //faster
-        init();
-        format = conf.get(MRUtils.FORMAT_PROP, format);
-        conf.set(MRUtils.FORMAT_PROP, format);
-        
-        String inputPath = conf.get(MRUtils.INPUT_PATH, args[0]);
-
-        Job job = new Job(conf);
-        job.setJarByClass(RdfFileInputTool.class);
-
-        // set up cloudbase input
-        job.setInputFormatClass(RdfFileInputFormat.class);
-        RdfFileInputFormat.addInputPath(job, new Path(inputPath));
-
-        // set input output of the particular job
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(RyaStatementWritable.class);
-
-        setupOutputFormat(job, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-
-        // set mapper and reducer classes
-        job.setMapperClass(StatementToMutationMapper.class);
-        job.setNumReduceTasks(0);
-
-        // Submit the job
-        Date startTime = new Date();
-        System.out.println("Job started: " + startTime);
-        int exitCode = job.waitForCompletion(true) ? 0 : 1;
-
-        if (exitCode == 0) {
-            Date end_time = new Date();
-            System.out.println("Job ended: " + end_time);
-            System.out.println("The job took "
-                    + (end_time.getTime() - startTime.getTime()) / 1000
-                    + " seconds.");
-            return job
-                    .getCounters()
-                    .findCounter("org.apache.hadoop.mapred.Task$Counter",
-                            "REDUCE_OUTPUT_RECORDS").getValue();
-        } else {
-            System.out.println("Job Failed!!!");
-        }
-
-        return -1;
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-        runJob(args);
-        return 0;
-    }
-
-    public static class StatementToMutationMapper extends Mapper<LongWritable, RyaStatementWritable, Text, Mutation> {
-        protected String tablePrefix;
-        protected Text spo_table;
-        protected Text po_table;
-        protected Text osp_table;
-        private byte[] cv = EMPTY_CV.getExpression();
-        RyaTableMutationsFactory mut;
-
-        public StatementToMutationMapper() {
-        }
-
-        @Override
-        protected void setup(Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-            mut = new RyaTableMutationsFactory(RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf)));
-            tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, RdfCloudTripleStoreConstants.TBL_PRFX_DEF);
-            spo_table = new Text(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-            po_table = new Text(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-            osp_table = new Text(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-
-            final String cv_s = conf.get(MRUtils.AC_CV_PROP);
-            if (cv_s != null)
-                cv = cv_s.getBytes();
-        }
-
-        @Override
-        protected void map(LongWritable key, RyaStatementWritable value, Context context) throws IOException, InterruptedException {
-            RyaStatement statement = value.getRyaStatement();
-            if (statement.getColumnVisibility() == null) {
-                statement.setColumnVisibility(cv);
-            }
-            Map<RdfCloudTripleStoreConstants.TABLE_LAYOUT, Collection<Mutation>> mutationMap =
-                    mut.serialize(statement);
-            Collection<Mutation> spo = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO);
-            Collection<Mutation> po = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.PO);
-            Collection<Mutation> osp = mutationMap.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP);
-
-            for (Mutation m : spo) {
-                context.write(spo_table, m);
-            }
-            for (Mutation m : po) {
-                context.write(po_table, m);
-            }
-            for (Mutation m : osp) {
-                context.write(osp_table, m);
-            }
-        }
-
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/upgrade/Upgrade322Tool.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/upgrade/Upgrade322Tool.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/upgrade/Upgrade322Tool.java
deleted file mode 100644
index 89f0aa5..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/upgrade/Upgrade322Tool.java
+++ /dev/null
@@ -1,240 +0,0 @@
-package mvm.rya.accumulo.mr.upgrade;
-
-/*
- * 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 mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-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.iterators.user.RegExFilter;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.calrissian.mango.types.LexiTypeEncoders;
-import org.calrissian.mango.types.TypeEncoder;
-
-import java.io.IOException;
-import java.util.Date;
-
-import static mvm.rya.api.RdfCloudTripleStoreConstants.*;
-
-/**
- */
-public class Upgrade322Tool extends AbstractAccumuloMRTool implements Tool {
-    @Override
-    public int run(String[] strings) throws Exception {
-        conf.set(MRUtils.JOB_NAME_PROP, "Upgrade to Rya 3.2.2");
-        //faster
-        init();
-
-        Job job = new Job(conf);
-        job.setJarByClass(Upgrade322Tool.class);
-
-        setupInputFormat(job);
-        AccumuloInputFormat.setInputTableName(job, tablePrefix + TBL_OSP_SUFFIX);
-
-        //we do not need to change any row that is a string, custom, or uri type
-        IteratorSetting regex = new IteratorSetting(30, "regex",
-                                                    RegExFilter.class);
-        RegExFilter.setRegexs(regex, "\\w*" + TYPE_DELIM + "[\u0003|\u0008|\u0002]", null, null, null, false);
-        RegExFilter.setNegate(regex, true);
-
-        // set input output of the particular job
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(Mutation.class);
-
-        setupOutputFormat(job, tablePrefix +
-                               TBL_SPO_SUFFIX);
-
-        // set mapper and reducer classes
-        job.setMapperClass(Upgrade322Mapper.class);
-        job.setReducerClass(Reducer.class);
-
-        // Submit the job
-        return job.waitForCompletion(true) ? 0 : 1;
-    }
-
-    public static void main(String[] args) {
-        try {
-            ToolRunner.run(new Configuration(), new Upgrade322Tool(), args);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    /**
-     * Reading from the OSP table
-     */
-    public static class Upgrade322Mapper extends Mapper<Key, Value, Text, Mutation> {
-
-        private String tablePrefix;
-        private Text spoTable;
-        private Text poTable;
-        private Text ospTable;
-
-        private final UpgradeObjectSerialization upgradeObjectSerialization;
-
-        public Upgrade322Mapper() {
-            this(new UpgradeObjectSerialization());
-        }
-
-        public Upgrade322Mapper(
-          UpgradeObjectSerialization upgradeObjectSerialization) {
-            this.upgradeObjectSerialization = upgradeObjectSerialization;
-        }
-
-        @Override
-        protected void setup(
-          Context context) throws IOException, InterruptedException {
-            super.setup(context);
-
-            tablePrefix = context.getConfiguration().get(
-              MRUtils.TABLE_PREFIX_PROPERTY, TBL_PRFX_DEF);
-            spoTable = new Text(tablePrefix + TBL_SPO_SUFFIX);
-            poTable = new Text(tablePrefix + TBL_PO_SUFFIX);
-            ospTable = new Text(tablePrefix + TBL_OSP_SUFFIX);
-        }
-
-        @Override
-        protected void map(
-          Key key, Value value, Context context)
-          throws IOException, InterruptedException {
-
-            //read the key, expect OSP
-            final String row = key.getRow().toString();
-            final int firstDelim = row.indexOf(DELIM);
-            final int secondDelim = row.indexOf(DELIM, firstDelim + 1);
-            final int typeDelim = row.lastIndexOf(TYPE_DELIM);
-            final String oldSerialization = row.substring(0, firstDelim);
-            char typeMarker = row.charAt(row.length() - 1);
-
-            final String subject = row.substring(firstDelim + 1, secondDelim);
-            final String predicate = row.substring(secondDelim + 1, typeDelim);
-            final String typeSuffix = TYPE_DELIM + typeMarker;
-
-            String newSerialization = upgradeObjectSerialization.upgrade(oldSerialization, typeMarker);
-            if(newSerialization == null) {
-                return;
-            }
-
-            //write out delete Mutations
-            Mutation deleteOldSerialization_osp = new Mutation(key.getRow());
-            deleteOldSerialization_osp.putDelete(key.getColumnFamily(), key.getColumnQualifier(),
-                               key.getColumnVisibilityParsed());
-            Mutation deleteOldSerialization_po = new Mutation(predicate + DELIM + oldSerialization + DELIM + subject + typeSuffix);
-            deleteOldSerialization_po.putDelete(key.getColumnFamily(),
-                                                key.getColumnQualifier(),
-                                                key.getColumnVisibilityParsed());
-            Mutation deleteOldSerialization_spo = new Mutation(subject + DELIM + predicate + DELIM + oldSerialization + typeSuffix);
-            deleteOldSerialization_spo.putDelete(key.getColumnFamily(), key.getColumnQualifier(),
-                                                key.getColumnVisibilityParsed());
-
-            //write out new serialization
-            Mutation putNewSerialization_osp = new Mutation(newSerialization + DELIM + subject + DELIM + predicate + typeSuffix);
-            putNewSerialization_osp.put(key.getColumnFamily(),
-                                        key.getColumnQualifier(),
-                                        key.getColumnVisibilityParsed(),
-                                        key.getTimestamp(), value);
-            Mutation putNewSerialization_po = new Mutation(predicate + DELIM + newSerialization + DELIM + subject + typeSuffix);
-            putNewSerialization_po.put(key.getColumnFamily(),
-                                       key.getColumnQualifier(),
-                                       key.getColumnVisibilityParsed(),
-                                       key.getTimestamp(), value);
-            Mutation putNewSerialization_spo = new Mutation(subject + DELIM + predicate + DELIM + newSerialization + typeSuffix);
-            putNewSerialization_spo.put(key.getColumnFamily(),
-                                        key.getColumnQualifier(),
-                                        key.getColumnVisibilityParsed(),
-                                        key.getTimestamp(), value);
-
-            //write out deletes to all tables
-            context.write(ospTable, deleteOldSerialization_osp);
-            context.write(poTable, deleteOldSerialization_po);
-            context.write(spoTable, deleteOldSerialization_spo);
-
-            //write out inserts to all tables
-            context.write(ospTable, putNewSerialization_osp);
-            context.write(poTable, putNewSerialization_po);
-            context.write(spoTable, putNewSerialization_spo);
-        }
-    }
-
-    public static class UpgradeObjectSerialization {
-
-        public static final TypeEncoder<Boolean, String>
-          BOOLEAN_STRING_TYPE_ENCODER = LexiTypeEncoders.booleanEncoder();
-        public static final TypeEncoder<Byte, String> BYTE_STRING_TYPE_ENCODER
-          = LexiTypeEncoders.byteEncoder();
-        public static final TypeEncoder<Date, String> DATE_STRING_TYPE_ENCODER
-          = LexiTypeEncoders.dateEncoder();
-        public static final TypeEncoder<Integer, String>
-          INTEGER_STRING_TYPE_ENCODER = LexiTypeEncoders.integerEncoder();
-        public static final TypeEncoder<Long, String> LONG_STRING_TYPE_ENCODER
-          = LexiTypeEncoders.longEncoder();
-        public static final TypeEncoder<Double, String>
-          DOUBLE_STRING_TYPE_ENCODER = LexiTypeEncoders.doubleEncoder();
-
-        public String upgrade(String object, int typeMarker) {
-            switch(typeMarker) {
-                case 10: //boolean
-                    final boolean bool = Boolean.parseBoolean(object);
-                    return BOOLEAN_STRING_TYPE_ENCODER.encode(bool);
-                case 9: //byte
-                    final byte b = Byte.parseByte(object);
-                    return BYTE_STRING_TYPE_ENCODER.encode(b);
-                case 4: //long
-                    final Long lng = Long.parseLong(object);
-                    return LONG_STRING_TYPE_ENCODER.encode(lng);
-                case 5: //int
-                    final Integer i = Integer.parseInt(object);
-                    return INTEGER_STRING_TYPE_ENCODER.encode(i);
-                case 6: //double
-                    String exp = object.substring(2, 5);
-                    char valueSign = object.charAt(0);
-                    char expSign = object.charAt(1);
-                    Integer expInt = Integer.parseInt(exp);
-                    if (expSign == '-') {
-                        expInt = 999 - expInt;
-                    }
-                    final String expDoubleStr =
-                      String.format("%s%sE%s%d", valueSign,
-                                    object.substring(6),
-                                    expSign, expInt);
-                    return DOUBLE_STRING_TYPE_ENCODER
-                      .encode(Double.parseDouble(expDoubleStr));
-                case 7: //datetime
-                    //check to see if it is an early release that includes the exact term xsd:dateTime
-                    final Long l = Long.MAX_VALUE - Long.parseLong(object);
-                    Date date = new Date(l);
-                    return DATE_STRING_TYPE_ENCODER.encode(date);
-                default:
-                    return null;
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloHDFSFileInputFormat.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloHDFSFileInputFormat.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloHDFSFileInputFormat.java
deleted file mode 100644
index c9dac6b..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloHDFSFileInputFormat.java
+++ /dev/null
@@ -1,206 +0,0 @@
-package mvm.rya.accumulo.mr.utils;
-
-/*
- * 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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-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.file.FileSKVIterator;
-import org.apache.accumulo.core.file.rfile.RFileOperations;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-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.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-
-/**
- * Finds the accumulo tablet files on the hdfs disk, and uses that as the input for MR jobs
- * Date: 5/11/12
- * Time: 2:04 PM
- */
-public class AccumuloHDFSFileInputFormat extends FileInputFormat<Key, Value> {
-
-    public static final Range ALLRANGE = new Range(new Text("\u0000"), new Text("\uFFFD"));
-
-    @Override
-    public List<InputSplit> getSplits(JobContext jobContext) throws IOException {
-        //read the params from AccumuloInputFormat
-        Configuration conf = jobContext.getConfiguration();
-        Instance instance = AccumuloProps.getInstance(jobContext);
-        String user = AccumuloProps.getUsername(jobContext);
-        AuthenticationToken password = AccumuloProps.getPassword(jobContext);
-        String table = AccumuloProps.getTablename(jobContext);
-        ArgumentChecker.notNull(instance);
-        ArgumentChecker.notNull(table);
-
-        //find the files necessary
-        try {
-        	AccumuloConfiguration acconf = instance.getConfiguration();
-            FileSystem fs = FileSystem.get(conf);
-            Connector connector = instance.getConnector(user, password);
-            TableOperations tos = connector.tableOperations();
-            String tableId = tos.tableIdMap().get(table);
-            String filePrefix = acconf.get(Property.INSTANCE_DFS_DIR) + "/tables/" + tableId;
-            System.out.println(filePrefix);
-
-            Scanner scanner = connector.createScanner("!METADATA", Constants.NO_AUTHS); //TODO: auths?
-            scanner.setRange(new Range(new Text(tableId + "\u0000"), new Text(tableId + "\uFFFD")));
-            scanner.fetchColumnFamily(new Text("file"));
-            List<String> files = new ArrayList<String>();
-            List<InputSplit> fileSplits = new ArrayList<InputSplit>();
-            Job job = new Job(conf);
-            for (Map.Entry<Key, Value> entry : scanner) {
-                String file = filePrefix + entry.getKey().getColumnQualifier().toString();
-                files.add(file);
-                Path path = new Path(file);
-                FileStatus fileStatus = fs.getFileStatus(path);
-                long len = fileStatus.getLen();
-                BlockLocation[] fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, len);
-                fileSplits.add(new FileSplit(path, 0, len, fileBlockLocations[0].getHosts()));
-//                FileInputFormat.addInputPath(job, path);
-            }
-            System.out.println(files);
-            return fileSplits;
-//            return super.getSplits(job);
-        } catch (Exception e) {
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public RecordReader<Key, Value> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-        return new RecordReader<Key, Value>() {
-
-            private FileSKVIterator fileSKVIterator;
-
-            @Override
-            public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-                FileSplit split = (FileSplit) inputSplit;
-                Configuration job = taskAttemptContext.getConfiguration();
-                Path file = split.getPath();
-//                long start = split.getStart();
-//                long length = split.getLength();
-                FileSystem fs = file.getFileSystem(job);
-//                FSDataInputStream fileIn = fs.open(file);
-//                System.out.println(start);
-//                if (start != 0L) {
-//                    fileIn.seek(start);
-//                }
-                Instance instance = AccumuloProps.getInstance(taskAttemptContext);
-
-                fileSKVIterator = RFileOperations.getInstance().openReader(file.toString(), ALLRANGE,
-                        new HashSet<ByteSequence>(), false, fs, job, instance.getConfiguration());
-//                fileSKVIterator = new RFileOperations2().openReader(fileIn, length - start, job);
-            }
-
-            @Override
-            public boolean nextKeyValue() throws IOException, InterruptedException {
-                fileSKVIterator.next();
-                return fileSKVIterator.hasTop();
-            }
-
-            @Override
-            public Key getCurrentKey() throws IOException, InterruptedException {
-                return fileSKVIterator.getTopKey();
-            }
-
-            @Override
-            public Value getCurrentValue() throws IOException, InterruptedException {
-                return fileSKVIterator.getTopValue();
-            }
-
-            @Override
-            public float getProgress() throws IOException, InterruptedException {
-                return 0;
-            }
-
-            @Override
-            public void close() throws IOException {
-                //To change body of implemented methods use File | Settings | File Templates.
-            }
-        };
-    }
-
-    public static void main(String[] args) {
-        try {
-            Job job = new Job(new Configuration());
-            job.setJarByClass(AccumuloHDFSFileInputFormat.class);
-            Configuration conf = job.getConfiguration();
-            conf.setBoolean("mapred.map.tasks.speculative.execution", false);
-            conf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
-            AccumuloInputFormat.setConnectorInfo(job, "root", new PasswordToken("secret"));
-            AccumuloInputFormat.setInputTableName(job, "l_spo");
-            AccumuloInputFormat.setScanAuthorizations(job, Constants.NO_AUTHS);
-            AccumuloInputFormat.setZooKeeperInstance(job, "acu13", "stratus25:2181");
-            AccumuloInputFormat.setRanges(job, Collections.singleton(ALLRANGE));
-            job.setMapperClass(NullMapper.class);
-            job.setNumReduceTasks(0);
-            job.setOutputFormatClass(NullOutputFormat.class);
-            if (args.length == 0) {
-                job.setInputFormatClass(AccumuloHDFSFileInputFormat.class);
-            } else {
-                job.setInputFormatClass(AccumuloInputFormat.class);
-            }
-            job.waitForCompletion(true);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    @SuppressWarnings("rawtypes")
-    public static class NullMapper extends Mapper {
-        @Override
-        protected void map(Object key, Object value, Context context) throws IOException, InterruptedException {
-
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloProps.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloProps.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloProps.java
deleted file mode 100644
index 2b89440..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/AccumuloProps.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package mvm.rya.accumulo.mr.utils;
-
-/*
- * 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 java.io.IOException;
-
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-@SuppressWarnings("rawtypes")
-public class AccumuloProps extends InputFormatBase {
-
-    @Override
-    public RecordReader createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-        throw new UnsupportedOperationException("Accumulo Props just holds properties");
-    }
-
-    public static Instance getInstance(JobContext  conf) {
-        return InputFormatBase.getInstance(conf);
-    }
-
-    public static AuthenticationToken getPassword(JobContext  conf) {
-        return InputFormatBase.getAuthenticationToken(conf);
-    }
-
-    public static String getUsername(JobContext conf) {
-        return InputFormatBase.getPrincipal(conf);
-    }
-
-    public static String getTablename(JobContext conf) {
-        return InputFormatBase.getInputTableName(conf);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/MRUtils.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/MRUtils.java b/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/MRUtils.java
deleted file mode 100644
index c3003d3..0000000
--- a/dao/accumulo.rya/src/main/java/mvm/rya/accumulo/mr/utils/MRUtils.java
+++ /dev/null
@@ -1,119 +0,0 @@
-package mvm.rya.accumulo.mr.utils;
-
-/*
- * 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 org.apache.hadoop.conf.Configuration;
-import org.openrdf.model.URI;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-/**
- * Class MRSailUtils
- * Date: May 19, 2011
- * Time: 10:34:06 AM
- */
-public class MRUtils {
-
-    public static final String JOB_NAME_PROP = "mapred.job.name";
-
-    public static final String AC_USERNAME_PROP = "ac.username";
-    public static final String AC_PWD_PROP = "ac.pwd";
-    public static final String AC_ZK_PROP = "ac.zk";
-    public static final String AC_INSTANCE_PROP = "ac.instance";
-    public static final String AC_TTL_PROP = "ac.ttl";
-    public static final String AC_TABLE_PROP = "ac.table";
-    public static final String AC_AUTH_PROP = "ac.auth";
-    public static final String AC_CV_PROP = "ac.cv";
-    public static final String AC_MOCK_PROP = "ac.mock";
-    public static final String AC_HDFS_INPUT_PROP = "ac.hdfsinput";
-    public static final String HADOOP_IO_SORT_MB = "ac.hdfsinput";
-    public static final String TABLE_LAYOUT_PROP = "rdf.tablelayout";
-    public static final String FORMAT_PROP = "rdf.format";
-    public static final String INPUT_PATH = "input";
-
-    public static final String NAMED_GRAPH_PROP = "rdf.graph";
-
-    public static final String TABLE_PREFIX_PROPERTY = "rdf.tablePrefix";
-
-    // rdf constants
-    public static final ValueFactory vf = new ValueFactoryImpl();
-    public static final URI RDF_TYPE = vf.createURI("http://www.w3.org/1999/02/22-rdf-syntax-ns#", "type");
-
-
-    // cloudbase map reduce utils
-
-//    public static Range retrieveRange(URI entry_key, URI entry_val) throws IOException {
-//        ByteArrayDataOutput startRowOut = ByteStreams.newDataOutput();
-//        startRowOut.write(RdfCloudTripleStoreUtils.writeValue(entry_key));
-//        if (entry_val != null) {
-//            startRowOut.write(RdfCloudTripleStoreConstants.DELIM_BYTES);
-//            startRowOut.write(RdfCloudTripleStoreUtils.writeValue(entry_val));
-//        }
-//        byte[] startrow = startRowOut.toByteArray();
-//        startRowOut.write(RdfCloudTripleStoreConstants.DELIM_STOP_BYTES);
-//        byte[] stoprow = startRowOut.toByteArray();
-//
-//        Range range = new Range(new Text(startrow), new Text(stoprow));
-//        return range;
-//    }
-
-
-    public static String getACTtl(Configuration conf) {
-        return conf.get(AC_TTL_PROP);
-    }
-
-    public static String getACUserName(Configuration conf) {
-        return conf.get(AC_USERNAME_PROP);
-    }
-
-    public static String getACPwd(Configuration conf) {
-        return conf.get(AC_PWD_PROP);
-    }
-
-    public static String getACZK(Configuration conf) {
-        return conf.get(AC_ZK_PROP);
-    }
-
-    public static String getACInstance(Configuration conf) {
-        return conf.get(AC_INSTANCE_PROP);
-    }
-
-    public static void setACUserName(Configuration conf, String str) {
-        conf.set(AC_USERNAME_PROP, str);
-    }
-
-    public static void setACPwd(Configuration conf, String str) {
-        conf.set(AC_PWD_PROP, str);
-    }
-
-    public static void setACZK(Configuration conf, String str) {
-        conf.set(AC_ZK_PROP, str);
-    }
-
-    public static void setACInstance(Configuration conf, String str) {
-        conf.set(AC_INSTANCE_PROP, str);
-    }
-
-    public static void setACTtl(Configuration conf, String str) {
-        conf.set(AC_TTL_PROP, str);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java b/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
deleted file mode 100644
index 1e74e7c..0000000
--- a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
+++ /dev/null
@@ -1,225 +0,0 @@
-package mvm.rya.accumulo.mr;
-/*
- * 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 mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRyaDAO;
-import mvm.rya.accumulo.RyaTableMutationsFactory;
-import mvm.rya.accumulo.mr.RyaStatementInputFormat.RyaStatementRecordReader;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.resolver.RyaTripleContext;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-public class RyaInputFormatTest {
-
-    static String username = "root", table = "rya_spo";
-    static PasswordToken password = new PasswordToken("");
-
-    static Instance instance;
-    static AccumuloRyaDAO apiImpl;
-
-    @BeforeClass
-    public static void init() throws Exception {
-        instance = new MockInstance("mock_instance");
-        Connector connector = instance.getConnector(username, password);
-        connector.tableOperations().create(table);
-
-        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
-        conf.setTablePrefix("rya_");
-        conf.setDisplayQueryPlan(false);
-
-        apiImpl = new AccumuloRyaDAO();
-        apiImpl.setConf(conf);
-        apiImpl.setConnector(connector);
-    }
-
-    @Before
-    public void before() throws Exception {
-        apiImpl.init();
-    }
-
-    @After
-    public void after() throws Exception {
-        apiImpl.dropAndDestroy();
-    }
-
-    @Test
-    public void testInputFormat() throws Exception {
-
-
-        RyaStatement input = RyaStatement.builder()
-            .setSubject(new RyaURI("http://www.google.com"))
-            .setPredicate(new RyaURI("http://some_other_uri"))
-            .setObject(new RyaURI("http://www.yahoo.com"))
-            .setColumnVisibility(new byte[0])
-            .setValue(new byte[0])
-            .build();
-
-        apiImpl.add(input);
-
-        Job jobConf = Job.getInstance();
-
-        RyaStatementInputFormat.setMockInstance(jobConf, instance.getInstanceName());
-        RyaStatementInputFormat.setConnectorInfo(jobConf, username, password);
-        RyaStatementInputFormat.setTableLayout(jobConf, TABLE_LAYOUT.SPO);
-
-        AccumuloInputFormat.setInputTableName(jobConf, table);
-        AccumuloInputFormat.setInputTableName(jobConf, table);
-        AccumuloInputFormat.setScanIsolation(jobConf, false);
-        AccumuloInputFormat.setLocalIterators(jobConf, false);
-        AccumuloInputFormat.setOfflineTableScan(jobConf, false);
-
-        RyaStatementInputFormat inputFormat = new RyaStatementInputFormat();
-
-        JobContext context = new JobContextImpl(jobConf.getConfiguration(), jobConf.getJobID());
-
-        List<InputSplit> splits = inputFormat.getSplits(context);
-
-        Assert.assertEquals(1, splits.size());
-
-        TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(context.getConfiguration(), new TaskAttemptID(new TaskID(), 1));
-
-        RecordReader reader = inputFormat.createRecordReader(splits.get(0), taskAttemptContext);
-
-        RyaStatementRecordReader ryaStatementRecordReader = (RyaStatementRecordReader)reader;
-        ryaStatementRecordReader.initialize(splits.get(0), taskAttemptContext);
-
-        List<RyaStatement> results = new ArrayList<RyaStatement>();
-        while(ryaStatementRecordReader.nextKeyValue()) {
-            RyaStatementWritable writable = ryaStatementRecordReader.getCurrentValue();
-            RyaStatement value = writable.getRyaStatement();
-            Text text = ryaStatementRecordReader.getCurrentKey();
-            RyaStatement stmt = RyaStatement.builder()
-                .setSubject(value.getSubject())
-                .setPredicate(value.getPredicate())
-                .setObject(value.getObject())
-                .setContext(value.getContext())
-                .setQualifier(value.getQualifer())
-                .setColumnVisibility(value.getColumnVisibility())
-                .setValue(value.getValue())
-                .build();
-            results.add(stmt);
-
-            System.out.println(text);
-            System.out.println(value);
-        }
-
-        Assert.assertTrue(results.size() == 2);
-        Assert.assertTrue(results.contains(input));
-    }
-
-    @Test
-    public void mapperTest() throws Exception {
-
-        RyaStatement input = RyaStatement.builder()
-            .setSubject(new RyaURI("http://www.google.com"))
-            .setPredicate(new RyaURI("http://some_other_uri"))
-            .setObject(new RyaURI("http://www.yahoo.com"))
-            .setValue(new byte[0])
-            .setTimestamp(0L)
-            .build();
-
-        RyaStatementWritable writable = new RyaStatementWritable();
-        writable.setRyaStatement(input);
-
-        RyaStatementMapper mapper = new RyaStatementMapper();
-        MapDriver<Text, RyaStatementWritable, Text, Mutation> mapDriver = MapDriver.newMapDriver(mapper);
-
-        RyaTripleContext context = RyaTripleContext.getInstance(new AccumuloRdfConfiguration());
-        RyaTableMutationsFactory mutationsFactory = new RyaTableMutationsFactory(context);
-
-        Map<TABLE_LAYOUT, Collection<Mutation>> mutations = mutationsFactory.serialize(input);
-
-        mapDriver.withInput(new Text("sometext"), writable);
-
-        for(TABLE_LAYOUT key : mutations.keySet()) {
-            Collection<Mutation> mutationCollection = mutations.get(key);
-            for(Mutation m : mutationCollection) {
-                mapDriver.withOutput(new Text("rya_" + key.name().toLowerCase()), m);
-            }
-        }
-
-        mapDriver.runTest(false);
-
-    }
-
-    @Test
-    public void reducerTest() throws Exception {
-        RyaStatement input = RyaStatement.builder()
-                .setSubject(new RyaURI("http://www.google.com"))
-                .setPredicate(new RyaURI("http://some_other_uri"))
-                .setObject(new RyaURI("http://www.yahoo.com"))
-                .setValue(new byte[0])
-                .setTimestamp(0L)
-                .build();
-
-        RyaStatementWritable writable = new RyaStatementWritable();
-        writable.setRyaStatement(input);
-
-        RyaStatementReducer reducer = new RyaStatementReducer();
-        ReduceDriver<WritableComparable, RyaStatementWritable, Text, Mutation> reduceDriver = ReduceDriver.newReduceDriver(reducer);
-
-        RyaTripleContext context = RyaTripleContext.getInstance(new AccumuloRdfConfiguration());
-        RyaTableMutationsFactory mutationsFactory = new RyaTableMutationsFactory(context);
-
-        Map<TABLE_LAYOUT, Collection<Mutation>> mutations = mutationsFactory.serialize(input);
-
-        reduceDriver.withInput(new Text("sometext"), Arrays.asList(writable));
-
-        for(TABLE_LAYOUT key : mutations.keySet()) {
-            Collection<Mutation> mutationCollection = mutations.get(key);
-            for(Mutation m : mutationCollection) {
-                reduceDriver.withOutput(new Text("rya_" + key.name().toLowerCase()), m);
-            }
-        }
-
-        reduceDriver.runTest(false);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountToolTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountToolTest.java b/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountToolTest.java
deleted file mode 100644
index bda73e2..0000000
--- a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/eval/AccumuloRdfCountToolTest.java
+++ /dev/null
@@ -1,282 +0,0 @@
-package mvm.rya.accumulo.mr.eval;
-
-/*
- * 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 mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRyaDAO;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.mock.MockInstance;
-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.security.TablePermission;
-import org.apache.hadoop.io.Text;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Created by IntelliJ IDEA.
- * Date: 4/24/12
- * Time: 5:05 PM
- * To change this template use File | Settings | File Templates.
- */
-@Ignore
-public class AccumuloRdfCountToolTest {
-
-    private String user = "user";
-    private String pwd = "pwd";
-    private String instance = "myinstance";
-    private String tablePrefix = "t_";
-    private Authorizations auths = Constants.NO_AUTHS;
-    private Connector connector;
-
-    private AccumuloRyaDAO dao;
-    private ValueFactory vf = new ValueFactoryImpl();
-    private AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
-    static String litdupsNS = "urn:test:litdups#";
-
-    @Before
-    public void setUp() throws Exception {
-        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-        SecurityOperations secOps = connector.securityOperations();
-        secOps.createUser(user, pwd.getBytes(), auths);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
-
-        dao = new AccumuloRyaDAO();
-        dao.setConnector(connector);
-        conf.setTablePrefix(tablePrefix);
-        dao.setConf(conf);
-        dao.init();
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        dao.destroy();
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-    }
-
-    @Test
-    public void testMR() throws Exception {
-        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
-        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10))));
-
-        AccumuloRdfCountTool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Drdf.tablePrefix=" + tablePrefix,
-        });
-
-        Map<String, Key> expectedValues = new HashMap<String, Key>();
-        String row = test1.getData();
-        expectedValues.put(row,
-                new Key(new Text(row),
-                        RdfCloudTripleStoreConstants.SUBJECT_CF_TXT,
-                        RdfCloudTripleStoreConstants.EMPTY_TEXT));
-        row = pred1.getData();
-        expectedValues.put(row,
-                new Key(new Text(row),
-                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
-                        RdfCloudTripleStoreConstants.EMPTY_TEXT));
-        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
-        scanner.setRange(new Range());
-        int count = 0;
-        for (Map.Entry<Key, Value> entry : scanner) {
-            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
-            assertEquals(11, Long.parseLong(entry.getValue().toString()));
-            count++;
-        }
-        assertEquals(2, count);
-    }
-
-//    public void testMRObject() throws Exception {
-//        URI pred1 = vf.createURI(litdupsNS, "pred1");
-//        Literal literal = vf.createLiteral(0);
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test0"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test1"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test2"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test3"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test4"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test5"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test6"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test7"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test8"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test9"), pred1, literal));
-//        dao.add(new StatementImpl(vf.createURI(litdupsNS, "test10"), pred1, literal));
-//        dao.commit();
-//
-//        AccumuloRdfCountTool.main(new String[]{
-//                "-Dac.mock=true",
-//                "-Dac.instance=" + instance,
-//                "-Dac.username=" + user,
-//                "-Dac.pwd=" + pwd,
-//                "-Drdf.tablePrefix=" + tablePrefix,
-//        });
-//
-//        Map<String, Key> expectedValues = new HashMap<String, Key>();
-//        byte[] row_bytes = RdfCloudTripleStoreUtils.writeValue(literal);
-//        expectedValues.put(new String(row_bytes),
-//                new Key(new Text(row_bytes),
-//                        RdfCloudTripleStoreConstants.OBJ_CF_TXT,
-//                        RdfCloudTripleStoreConstants.INFO_TXT));
-//        row_bytes = RdfCloudTripleStoreUtils.writeValue(pred1);
-//        expectedValues.put(new String(row_bytes),
-//                new Key(new Text(row_bytes),
-//                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
-//                        RdfCloudTripleStoreConstants.INFO_TXT));
-//        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
-//        scanner.setRange(new Range());
-//        int count = 0;
-//        for (Map.Entry<Key, Value> entry : scanner) {
-//            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
-//            assertEquals(11, Long.parseLong(entry.getValue().toString()));
-//            count++;
-//        }
-//        assertEquals(2, count);
-//    }
-
-    @Test
-    public void testTTL() throws Exception {
-        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
-        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9))));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10))));
-
-        AccumuloRdfCountTool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Dac.ttl=0",
-                "-Drdf.tablePrefix=" + tablePrefix,
-        });
-
-        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
-        scanner.setRange(new Range());
-        int count = 0;
-        for (Map.Entry<Key, Value> entry : scanner) {
-            count++;
-        }
-        assertEquals(0, count);
-    }
-
-    @Test
-    public void testContext() throws Exception {
-        RyaURI test1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "test1"));
-        RyaURI pred1 = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "pred1"));
-        RyaURI cntxt = RdfToRyaConversions.convertURI(vf.createURI(litdupsNS, "cntxt"));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(0)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(1)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(2)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(3)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(4)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(5)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(6)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(7)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(8)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(9)), cntxt));
-        dao.add(new RyaStatement(test1, pred1, RdfToRyaConversions.convertLiteral(vf.createLiteral(10)), cntxt));
-
-        AccumuloRdfCountTool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Drdf.tablePrefix=" + tablePrefix,
-        });
-
-        Map<String, Key> expectedValues = new HashMap<String, Key>();
-        String row = test1.getData();
-        expectedValues.put(row,
-                new Key(new Text(row),
-                        RdfCloudTripleStoreConstants.SUBJECT_CF_TXT,
-                        new Text(cntxt.getData())));
-        row = pred1.getData();
-        expectedValues.put(row,
-                new Key(new Text(row),
-                        RdfCloudTripleStoreConstants.PRED_CF_TXT,
-                        new Text(cntxt.getData())));
-        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, auths);
-        scanner.setRange(new Range());
-        int count = 0;
-        for (Map.Entry<Key, Value> entry : scanner) {
-            assertTrue(expectedValues.get(entry.getKey().getRow().toString()).equals(entry.getKey(), PartialKey.ROW_COLFAM_COLQUAL));
-            assertEquals(11, Long.parseLong(entry.getValue().toString()));
-            count++;
-        }
-        assertEquals(2, count);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputToolTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputToolTest.java b/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputToolTest.java
deleted file mode 100644
index 02b8357..0000000
--- a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/fileinput/RdfFileInputToolTest.java
+++ /dev/null
@@ -1,146 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 java.util.Iterator;
-import java.util.Map;
-
-import junit.framework.TestCase;
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.resolver.RyaContext;
-import mvm.rya.api.resolver.RyaTripleContext;
-import mvm.rya.api.resolver.triple.TripleRow;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.mock.MockInstance;
-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.core.security.TablePermission;
-import org.apache.hadoop.io.Text;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.rio.RDFFormat;
-
-/**
- * Created by IntelliJ IDEA.
- * Date: 4/25/12
- * Time: 10:51 AM
- * To change this template use File | Settings | File Templates.
- */
-public class RdfFileInputToolTest extends TestCase {
-
-    private String user = "user";
-    private String pwd = "pwd";
-    private String instance = "myinstance";
-    private String tablePrefix = "t_";
-    private Authorizations auths = Constants.NO_AUTHS;
-    private Connector connector;
-
-    @Override
-    public void setUp() throws Exception {
-        super.setUp();
-        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-        SecurityOperations secOps = connector.securityOperations();
-        secOps.createUser(user, pwd.getBytes(), auths);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
-    }
-
-    @Override
-    public void tearDown() throws Exception {
-        super.tearDown();
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-    }
-
-    public void testNTriplesInput() throws Exception {
-        RdfFileInputTool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Drdf.tablePrefix=" + tablePrefix,
-                "-Drdf.format=" + RDFFormat.NTRIPLES.getName(),
-                "src/test/resources/test.ntriples",
-        });
-
-        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, auths);
-        scanner.setRange(new Range());
-        Iterator<Map.Entry<Key, Value>> iterator = scanner.iterator();
-        ValueFactory vf = new ValueFactoryImpl();
-        assertTrue(iterator.hasNext());
-        RyaStatement rs = new RyaStatement(new RyaURI("urn:lubm:rdfts#GraduateStudent01"),
-                new RyaURI("urn:lubm:rdfts#hasFriend"),
-                new RyaURI("urn:lubm:rdfts#GraduateStudent02"));
-        assertEquals(new Text(RyaTripleContext.getInstance(new AccumuloRdfConfiguration()).serializeTriple(rs).get(TABLE_LAYOUT.SPO).getRow()), iterator.next().getKey().getRow());
-    }
-
-    public void testInputContext() throws Exception {
-        RdfFileInputTool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Drdf.tablePrefix=" + tablePrefix,
-                "-Drdf.format=" + RDFFormat.TRIG.getName(),
-                "src/test/resources/namedgraphs.trig",
-        });
-
-        Scanner scanner = connector.createScanner(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, auths);
-        scanner.setRange(new Range());
-        Iterator<Map.Entry<Key, Value>> iterator = scanner.iterator();
-        ValueFactory vf = new ValueFactoryImpl();
-        assertTrue(iterator.hasNext());
-        RyaStatement rs = new RyaStatement(new RyaURI("http://www.example.org/exampleDocument#Monica"),
-                new RyaURI("http://www.example.org/vocabulary#name"),
-                new RyaType("Monica Murphy"),
-                new RyaURI("http://www.example.org/exampleDocument#G1"));
-        Key key = iterator.next().getKey();
-
-        TripleRow tripleRow = RyaTripleContext.getInstance(new AccumuloRdfConfiguration()).serializeTriple(rs).get(TABLE_LAYOUT.SPO);
-        assertEquals(new Text(tripleRow.getRow()), key.getRow());
-        assertEquals(new Text(tripleRow.getColumnFamily()), key.getColumnFamily());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/Upgrade322ToolTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/Upgrade322ToolTest.java b/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/Upgrade322ToolTest.java
deleted file mode 100644
index 5ac2d74..0000000
--- a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/Upgrade322ToolTest.java
+++ /dev/null
@@ -1,319 +0,0 @@
-package mvm.rya.accumulo.mr.upgrade;
-
-/*
- * 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 junit.framework.TestCase;
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRyaDAO;
-import mvm.rya.accumulo.query.AccumuloRyaQueryEngine;
-import mvm.rya.api.RdfCloudTripleStoreConstants;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.api.persist.RyaDAOException;
-import mvm.rya.api.persist.query.RyaQuery;
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.*;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.mock.MockInstance;
-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.core.security.TablePermission;
-import org.calrissian.mango.collect.CloseableIterable;
-import org.openrdf.model.vocabulary.XMLSchema;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * Created by IntelliJ IDEA.
- * Date: 4/25/12
- * Time: 10:51 AM
- * To change this template use File | Settings | File Templates.
- */
-public class Upgrade322ToolTest extends TestCase {
-
-    private String user = "user";
-    private String pwd = "pwd";
-    private String instance = "myinstance";
-    private String tablePrefix = "t_";
-    private Authorizations auths = Constants.NO_AUTHS;
-    private Connector connector;
-
-    @Override
-    public void setUp() throws Exception {
-        super.setUp();
-
-        final String spoTable = tablePrefix +
-                                RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX;
-        final String poTable = tablePrefix +
-                               RdfCloudTripleStoreConstants.TBL_PO_SUFFIX;
-        final String ospTable = tablePrefix +
-                                RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX;
-
-        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
-
-        connector.tableOperations().create(spoTable);
-        connector.tableOperations().create(poTable);
-        connector.tableOperations().create(ospTable);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-        SecurityOperations secOps = connector.securityOperations();
-        secOps.createUser(user, pwd.getBytes(), auths);
-        secOps.grantTablePermission(user, spoTable, TablePermission.READ);
-        secOps.grantTablePermission(user, poTable, TablePermission.READ);
-        secOps.grantTablePermission(user, ospTable, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.READ);
-        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
-
-        //load data
-        final BatchWriter ospWriter = connector
-          .createBatchWriter(ospTable, new BatchWriterConfig());
-        ospWriter.addMutation(getMutation("00000000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010/tracked-data-provenance/ns#longLit\u0001\u0004"));
-        ospWriter.addMutation(getMutation("00000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#intLit\u0001\u0005"));
-        ospWriter.addMutation(getMutation("00000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#byteLit\u0001\t"));
-        ospWriter.addMutation(getMutation("00001 1.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#doubleLit\u0001\u0006"));
-        ospWriter.addMutation(getMutation("10\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0000http" +
-        "://here/2010/tracked-data-provenance/ns#shortLit\u0001http://www.w3" +
-        ".org/2001/XMLSchema#short\u0001\b"));
-        ospWriter.addMutation(getMutation("10.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#floatLit\u0001http" +
-        "://www.w3.org/2001/XMLSchema#float\u0001\b"));
-        ospWriter.addMutation(getMutation("3.0.0\u0000urn:mvm.rya/2012/05#rts\u0000urn:mvm" +
-        ".rya/2012/05#version\u0001\u0003"));
-        ospWriter.addMutation(getMutation("9223370726404375807\u0000http://here/2010/tracked-data-provenance/ns" +
-        "#uuid10\u0000http://here/2010/tracked-data-provenance/ns#dateLit" +
-        "\u0001\u0007"));
-        ospWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#Created\u0000http://here" +
-        "/2010/tracked-data-provenance/ns#uuid10\u0000http://www.w3" +
-        ".org/1999/02/22-rdf-syntax-ns#type\u0001\u0002"));
-        ospWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#objectuuid1\u0000http" +
-        "://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010" +
-        "/tracked-data-provenance/ns#uriLit\u0001\u0002"));
-        ospWriter.addMutation(getMutation("stringLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#stringLit\u0001" +
-        "\u0003"));
-        ospWriter.addMutation(getMutation("true\u0000http://here/2010/tracked-data-provenance/ns#uuid10" +
-        "\u0000http://here/2010/tracked-data-provenance/ns#booleanLit\u0001\n"));
-        ospWriter.flush();
-        ospWriter.close();
-
-        final BatchWriter spoWriter = connector
-          .createBatchWriter(spoTable, new BatchWriterConfig());
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010/tracked-data-provenance/ns#longLit\u000000000000000000000010\u0001\u0004"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#intLit\u000000000000010\u0001\u0005"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#byteLit\u000000000010\u0001\t"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#doubleLit\u000000001 1.0\u0001\u0006"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10\u0000http" +
-                                          "://here/2010/tracked-data-provenance/ns#shortLit\u000010\u0001http://www.w3" +
-                                          ".org/2001/XMLSchema#short\u0001\b"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#floatLit\u0001http" +
-                                          "://www.w3.org/2001/XMLSchema#float\u000010.0\u0001\b"));
-        spoWriter.addMutation(getMutation("urn:mvm.rya/2012/05#rts\u0000urn:mvm" +
-                                          ".rya/2012/05#version\u00003.0.0\u0001\u0003"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns" +
-                                          "#uuid10\u0000http://here/2010/tracked-data-provenance/ns#dateLit" +
-                                          "\u00009223370726404375807\u0001\u0007"));
-        spoWriter.addMutation(getMutation("http://here" +
-                                          "/2010/tracked-data-provenance/ns#uuid10\u0000http://www.w3" +
-                                          ".org/1999/02/22-rdf-syntax-ns#type\u0000http://here/2010/tracked-data-provenance/ns#Created\u0001\u0002"));
-        spoWriter.addMutation(getMutation("http" +
-                                          "://here/2010/tracked-data-provenance/ns#uuid10\u0000http://here/2010" +
-                                          "/tracked-data-provenance/ns#uriLit\u0000http://here/2010/tracked-data-provenance/ns#objectuuid1\u0001\u0002"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#stringLit\u0000stringLit\u0001" +
-                                          "\u0003"));
-        spoWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#uuid10" +
-                                          "\u0000http://here/2010/tracked-data-provenance/ns#booleanLit\u0000true\u0001\n"));
-        spoWriter.flush();
-        spoWriter.close();
-
-        final BatchWriter poWriter = connector
-          .createBatchWriter(poTable, new BatchWriterConfig());
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#longLit\u000000000000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0004"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#intLit\u000000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0005"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#byteLit\u000000000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\t"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#doubleLit\u000000001 1.0\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0006"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#shortLit\u000010\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001http://www.w3" +
-                                          ".org/2001/XMLSchema#short\u0001\b"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#floatLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001http" +
-                                          "://www.w3.org/2001/XMLSchema#float\u000010.0\u0001\b"));
-        poWriter.addMutation(getMutation("urn:mvm" +
-                                          ".rya/2012/05#version\u00003.0.0\u0000urn:mvm.rya/2012/05#rts\u0001\u0003"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#dateLit" +
-                                          "\u00009223370726404375807\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0007"));
-        poWriter.addMutation(getMutation("http://www.w3" +
-                                          ".org/1999/02/22-rdf-syntax-ns#type\u0000http://here/2010/tracked-data-provenance/ns#Created\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0002"));
-        poWriter.addMutation(getMutation("http://here/2010" +
-                                          "/tracked-data-provenance/ns#uriLit\u0000http://here/2010/tracked-data-provenance/ns#objectuuid1\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\u0002"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#stringLit\u0000stringLit\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001" +
-                                          "\u0003"));
-        poWriter.addMutation(getMutation("http://here/2010/tracked-data-provenance/ns#booleanLit\u0000true\u0000http://here/2010/tracked-data-provenance/ns#uuid10\u0001\n"));
-        poWriter.flush();
-        poWriter.close();
-    }
-
-    public Mutation getMutation(String row) {
-        final Mutation mutation = new Mutation(row);
-        mutation.put("", "", "");
-        return mutation;
-    }
-
-    @Override
-    public void tearDown() throws Exception {
-        super.tearDown();
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
-        connector.tableOperations().delete(
-          tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
-        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
-        connector.tableOperations().delete(
-          tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
-    }
-
-    public void testUpgrade() throws Exception {
-        Upgrade322Tool.main(new String[]{
-                "-Dac.mock=true",
-                "-Dac.instance=" + instance,
-                "-Dac.username=" + user,
-                "-Dac.pwd=" + pwd,
-                "-Drdf.tablePrefix=" + tablePrefix,
-        });
-
-        final AccumuloRdfConfiguration configuration = new AccumuloRdfConfiguration();
-        configuration.setTablePrefix(tablePrefix);
-        final AccumuloRyaDAO ryaDAO = new AccumuloRyaDAO();
-        ryaDAO.setConnector(connector);
-        ryaDAO.setConf(configuration);
-        ryaDAO.init();
-
-        final AccumuloRyaQueryEngine queryEngine = ryaDAO.getQueryEngine();
-
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#booleanLit"),
-          new RyaType(XMLSchema.BOOLEAN, "true")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#longLit"),
-          new RyaType(XMLSchema.LONG, "10")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#intLit"),
-          new RyaType(XMLSchema.INTEGER, "10")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#byteLit"),
-          new RyaType(XMLSchema.BYTE, "10")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#doubleLit"),
-          new RyaType(XMLSchema.DOUBLE, "10.0")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#dateLit"),
-          new RyaType(XMLSchema.DATETIME, "2011-07-12T06:00:00.000Z")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#stringLit"),
-          new RyaType("stringLit")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uuid10"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns#uriLit"),
-          new RyaURI("http://here/2010/tracked-data-provenance/ns" +
-                     "#objectuuid1")), queryEngine);
-        verify(new RyaStatement(
-          new RyaURI("urn:mvm.rya/2012/05#rts"),
-          new RyaURI("urn:mvm.rya/2012/05#version"),
-          new RyaType("3.0.0")), queryEngine);
-    }
-
-    private void verify(RyaStatement ryaStatement, AccumuloRyaQueryEngine queryEngine)
-      throws RyaDAOException, IOException {
-
-        //check osp
-        CloseableIterable<RyaStatement> statements =
-          queryEngine.query(RyaQuery.builder(new RyaStatement(null, null, ryaStatement.getObject()))
-                                    .build());
-        try {
-            verifyFirstStatement(ryaStatement, statements);
-        } finally {
-            statements.close();
-        }
-
-        //check po
-        statements = queryEngine.query(RyaQuery.builder(
-          new RyaStatement(null, ryaStatement.getPredicate(),
-                           ryaStatement.getObject())).build());
-        try {
-            verifyFirstStatement(ryaStatement, statements);
-        } finally {
-            statements.close();
-        }
-
-        //check spo
-        statements = queryEngine.query(RyaQuery.builder(
-          new RyaStatement(ryaStatement.getSubject(),
-                           ryaStatement.getPredicate(),
-                           ryaStatement.getObject())).build());
-        try {
-            verifyFirstStatement(ryaStatement, statements);
-        } finally {
-            statements.close();
-        }
-    }
-
-    private void verifyFirstStatement(
-      RyaStatement ryaStatement, CloseableIterable<RyaStatement> statements) {
-        final Iterator<RyaStatement> iterator = statements.iterator();
-        assertTrue(iterator.hasNext());
-        final RyaStatement first = iterator.next();
-        assertEquals(ryaStatement.getSubject(), first.getSubject());
-        assertEquals(ryaStatement.getPredicate(), first.getPredicate());
-        assertEquals(ryaStatement.getObject(), first.getObject());
-        assertFalse(iterator.hasNext());
-    }
-
-    public void printTableData(String tableName)
-      throws TableNotFoundException{
-        Scanner scanner = connector.createScanner(tableName, auths);
-        scanner.setRange(new Range());
-        for(Map.Entry<Key, Value> entry : scanner) {
-            final Key key = entry.getKey();
-            final Value value = entry.getValue();
-            System.out.println(key.getRow() + " " + key.getColumnFamily() + " " + key.getColumnQualifier() + " " + key.getTimestamp() + " " + value.toString());
-        }
-    }
-
-}



[3/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Posted by pu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
new file mode 100644
index 0000000..88f9030
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/AbstractAccumuloMRTool.java
@@ -0,0 +1,305 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 mvm.rya.accumulo.AccumuloRdfConstants;
+import mvm.rya.api.RdfCloudTripleStoreConfiguration;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
+import mvm.rya.api.RdfCloudTripleStoreUtils;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+
+import java.io.IOException;
+
+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.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.iterators.user.AgeOffFilter;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.openrdf.rio.RDFFormat;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Base class for MapReduce tools that interact with Accumulo-backed Rya. Holds
+ * a {@link Configuration} to keep track of connection parameters.
+ * <p>
+ * Can be configured to read input from Rya, either as
+ * {@link RyaStatementWritable}s or as Accumulo rows, or to read statements from
+ * RDF files.
+ * <p>
+ * Can be configured to send output either by inserting RyaStatementWritables to
+ * a Rya instance, or by writing arbitrary
+ * {@link org.apache.accumulo.core.data.Mutation}s directly to Accumulo tables.
+ */
+public abstract class AbstractAccumuloMRTool implements Tool {
+    static int DEFAULT_IO_SORT_MB = 256;
+
+    protected Configuration conf;
+
+    // Connection parameters
+    protected String zk;
+    protected String instance;
+    protected String userName;
+    protected String pwd;
+    protected Authorizations authorizations;
+    protected boolean mock = false;
+    protected boolean hdfsInput = false;
+    protected String ttl;
+    protected String tablePrefix;
+    protected TABLE_LAYOUT rdfTableLayout;
+
+    /**
+     * Gets the Configuration containing any relevant options.
+     * @return This Tool's Configuration object.
+     */
+    @Override
+    public Configuration getConf() {
+        return conf;
+    }
+
+    /**
+     * Set this Tool's Configuration.
+     */
+    @Override
+    public void setConf(Configuration configuration) {
+        this.conf = configuration;
+    }
+
+    /**
+     * Initializes configuration parameters, checking that required parameters
+     * are found and ensuring that options corresponding to multiple property
+     * names are set consistently. Requires at least that the username,
+     * password, and instance name are all configured. Zookeeper hosts must be
+     * configured if not using a mock instance. Table prefix, if not provided,
+     * will be set to {@link RdfCloudTripleStoreConstants#TBL_PRFX_DEF}. Should
+     * be called before configuring input/output. See {@link MRUtils} for
+     * configuration properties.
+     */
+    protected void init() {
+        // Load configuration parameters
+        zk = MRUtils.getACZK(conf);
+        instance = MRUtils.getACInstance(conf);
+        userName = MRUtils.getACUserName(conf);
+        pwd = MRUtils.getACPwd(conf);
+        mock = MRUtils.getACMock(conf, false);
+        ttl = MRUtils.getACTtl(conf);
+        tablePrefix = MRUtils.getTablePrefix(conf);
+        rdfTableLayout = MRUtils.getTableLayout(conf, TABLE_LAYOUT.OSP);
+        hdfsInput = conf.getBoolean(MRUtils.AC_HDFS_INPUT_PROP, false);
+        // Set authorizations if specified
+        String authString = conf.get(MRUtils.AC_AUTH_PROP);
+        if (authString != null && !authString.isEmpty()) {
+            authorizations = new Authorizations(authString.split(","));
+            conf.set(ConfigUtils.CLOUDBASE_AUTHS, authString); // for consistency
+        }
+        else {
+            authorizations = AccumuloRdfConstants.ALL_AUTHORIZATIONS;
+        }
+        // Set table prefix to the default if not set
+        if (tablePrefix == null) {
+            tablePrefix = RdfCloudTripleStoreConstants.TBL_PRFX_DEF;
+            MRUtils.setTablePrefix(conf, tablePrefix);
+        }
+        // Check for required configuration parameters
+        Preconditions.checkNotNull(instance, "Accumulo instance name [" + MRUtils.AC_INSTANCE_PROP + "] not set.");
+        Preconditions.checkNotNull(userName, "Accumulo username [" + MRUtils.AC_USERNAME_PROP + "] not set.");
+        Preconditions.checkNotNull(pwd, "Accumulo password [" + MRUtils.AC_PWD_PROP + "] not set.");
+        Preconditions.checkNotNull(tablePrefix, "Table prefix [" + MRUtils.TABLE_PREFIX_PROPERTY + "] not set.");
+        RdfCloudTripleStoreConstants.prefixTables(tablePrefix);
+        // If connecting to real accumulo, set additional parameters and require zookeepers
+        if (!mock) {
+            Preconditions.checkNotNull(zk, "Zookeeper hosts not set (" + MRUtils.AC_ZK_PROP + ")");
+            conf.setBoolean("mapred.map.tasks.speculative.execution", false);
+            conf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
+            if (conf.get(MRUtils.HADOOP_IO_SORT_MB) == null) {
+                conf.setInt(MRUtils.HADOOP_IO_SORT_MB, DEFAULT_IO_SORT_MB);
+            }
+            conf.set(ConfigUtils.CLOUDBASE_ZOOKEEPERS, zk); // for consistency
+        }
+        // Ensure consistency between alternative configuration properties
+        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, instance);
+        conf.set(ConfigUtils.CLOUDBASE_USER, userName);
+        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, pwd);
+        conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, mock);
+        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, tablePrefix);
+    }
+
+    /**
+     * Sets up Accumulo input for a job: the job receives
+     * ({@link org.apache.accumulo.core.data.Key},
+     * {@link org.apache.accumulo.core.data.Value}) pairs from the table
+     * specified by the configuration (using
+     * {@link MRUtils#TABLE_PREFIX_PROPERTY} and
+     * {@link MRUtils#TABLE_LAYOUT_PROP}).
+     * @param   job     MapReduce Job to configure
+     * @throws  AccumuloSecurityException if connecting to Accumulo with the
+     *          given username and password fails.
+     */
+    protected void setupAccumuloInput(Job job) throws AccumuloSecurityException {
+        // set up accumulo input
+        if (!hdfsInput) {
+            job.setInputFormatClass(AccumuloInputFormat.class);
+        } else {
+            job.setInputFormatClass(AccumuloHDFSFileInputFormat.class);
+        }
+        AccumuloInputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd));
+        String tableName = RdfCloudTripleStoreUtils.layoutPrefixToTable(rdfTableLayout, tablePrefix);
+        AccumuloInputFormat.setInputTableName(job, tableName);
+        AccumuloInputFormat.setScanAuthorizations(job, authorizations);
+        if (mock) {
+            AccumuloInputFormat.setMockInstance(job, instance);
+        } else {
+            ClientConfiguration clientConfig = ClientConfiguration.loadDefault()
+                    .withInstance(instance).withZkHosts(zk);
+            AccumuloInputFormat.setZooKeeperInstance(job, clientConfig);
+        }
+        if (ttl != null) {
+            IteratorSetting setting = new IteratorSetting(1, "fi", AgeOffFilter.class.getName());
+            AgeOffFilter.setTTL(setting, Long.valueOf(ttl));
+            AccumuloInputFormat.addIterator(job, setting);
+        }
+    }
+
+    /**
+     * Sets up Rya input for a job: the job receives
+     * ({@link org.apache.hadoop.io.LongWritable}, {@link RyaStatementWritable})
+     * pairs from a Rya instance. Uses the same configuration properties to
+     * connect as direct Accumulo input, but returns statement data instead of
+     * row data.
+     * @param   job     Job to configure
+     * @throws  AccumuloSecurityException if connecting to Accumulo with the
+     *          given username and password fails.
+     */
+    protected void setupRyaInput(Job job) throws AccumuloSecurityException {
+        setupAccumuloInput(job);
+        job.setInputFormatClass(RyaInputFormat.class);
+    }
+
+    /**
+     * Sets up RDF file input for a job: the job receives
+     * ({@link org.apache.hadoop.io.LongWritable}, {@link RyaStatementWritable})
+     * pairs from RDF file(s) found at the specified path.
+     * @param   job   Job to configure
+     * @param   inputPath     File or directory name
+     * @param   defaultFormat  Default RDF serialization format, can be
+     *                         overridden by {@link MRUtils#FORMAT_PROP}
+     * @throws  IOException if there's an error interacting with the
+     *          {@link org.apache.hadoop.fs.FileSystem}.
+     */
+    protected void setupFileInput(Job job, String inputPath, RDFFormat defaultFormat) throws IOException {
+        RDFFormat format = MRUtils.getRDFFormat(conf);
+        if (format == null) {
+            format = defaultFormat;
+        }
+        RdfFileInputFormat.addInputPath(job, new Path(inputPath));
+        RdfFileInputFormat.setRDFFormat(job, format);
+        job.setInputFormatClass(RdfFileInputFormat.class);
+    }
+
+    /**
+     * Sets up Accumulo output for a job: allows the job to write (String,
+     * Mutation) pairs, where the Mutation will be written to the table named by
+     * the String.
+     * @param   job Job to configure
+     * @param   outputTable Default table to send output to
+     * @throws  AccumuloSecurityException if connecting to Accumulo with the
+     *          given username and password fails
+     */
+    protected void setupAccumuloOutput(Job job, String outputTable) throws AccumuloSecurityException {
+        AccumuloOutputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd));
+        AccumuloOutputFormat.setCreateTables(job, true);
+        AccumuloOutputFormat.setDefaultTableName(job, outputTable);
+        if (mock) {
+            AccumuloOutputFormat.setMockInstance(job, instance);
+        } else {
+            ClientConfiguration clientConfig = ClientConfiguration.loadDefault()
+                    .withInstance(instance).withZkHosts(zk);
+            AccumuloOutputFormat.setZooKeeperInstance(job, clientConfig);
+        }
+        job.setOutputFormatClass(AccumuloOutputFormat.class);
+    }
+
+    /**
+     * Sets up Rya output for a job: allows the job to write
+     * {@link RyaStatementWritable} data, which will in turn be input into the
+     * configured Rya instance. To perform secondary indexing, use the
+     * configuration variables in {@link ConfigUtils}.
+     * @param   job Job to configure
+     * @throws  AccumuloSecurityException if connecting to Accumulo with the
+     *          given username and password fails
+     */
+    protected void setupRyaOutput(Job job) throws AccumuloSecurityException {
+        job.setOutputFormatClass(RyaOutputFormat.class);
+        job.setOutputValueClass(RyaStatementWritable.class);
+        // Specify default visibility of output rows, if given
+        RyaOutputFormat.setDefaultVisibility(job, conf.get(MRUtils.AC_CV_PROP));
+        // Specify named graph, if given
+        RyaOutputFormat.setDefaultContext(job, conf.get(MRUtils.NAMED_GRAPH_PROP));
+        // Set the output prefix
+        RyaOutputFormat.setTablePrefix(job, tablePrefix);
+        // Determine which indexers to use based on the config
+        RyaOutputFormat.setFreeTextEnabled(job,  ConfigUtils.getUseFreeText(conf));
+        RyaOutputFormat.setGeoEnabled(job,  ConfigUtils.getUseGeo(conf));
+        RyaOutputFormat.setTemporalEnabled(job,  ConfigUtils.getUseTemporal(conf));
+        RyaOutputFormat.setEntityEnabled(job,  ConfigUtils.getUseEntity(conf));
+        // Configure the Accumulo connection
+        AccumuloOutputFormat.setConnectorInfo(job, userName, new PasswordToken(pwd));
+        AccumuloOutputFormat.setCreateTables(job, true);
+        AccumuloOutputFormat.setDefaultTableName(job, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        if (mock) {
+            RyaOutputFormat.setMockInstance(job, instance);
+        } else {
+            ClientConfiguration clientConfig = ClientConfiguration.loadDefault()
+                    .withInstance(instance).withZkHosts(zk);
+            AccumuloOutputFormat.setZooKeeperInstance(job, clientConfig);
+        }
+    }
+
+    /**
+     * Connects to Accumulo, using the stored connection parameters.
+     * @return  A Connector to an Accumulo instance, which could be a mock
+     *          instance.
+     * @throws AccumuloException if connecting to Accumulo fails.
+     * @throws AccumuloSecurityException if authenticating with Accumulo fails.
+     */
+    protected Connector getConnector() throws AccumuloSecurityException, AccumuloException {
+        Instance zooKeeperInstance;
+        if (mock) {
+            zooKeeperInstance = new MockInstance(instance);
+        }
+        else {
+            zooKeeperInstance = new ZooKeeperInstance(instance, zk);
+        }
+        return zooKeeperInstance.getConnector(userName, new PasswordToken(pwd));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/AccumuloHDFSFileInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/AccumuloHDFSFileInputFormat.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/AccumuloHDFSFileInputFormat.java
new file mode 100644
index 0000000..90461d1
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/AccumuloHDFSFileInputFormat.java
@@ -0,0 +1,161 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+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.file.FileSKVIterator;
+import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+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.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+/**
+ * {@link FileInputFormat} that finds the Accumulo tablet files on the HDFS
+ * disk, and uses that as the input for MapReduce jobs.
+ */
+public class AccumuloHDFSFileInputFormat extends FileInputFormat<Key, Value> {
+
+    public static final Range ALLRANGE = new Range(new Text("\u0000"), new Text("\uFFFD"));
+
+    @Override
+    public List<InputSplit> getSplits(JobContext jobContext) throws IOException {
+        //read the params from AccumuloInputFormat
+        Configuration conf = jobContext.getConfiguration();
+        Instance instance = MRUtils.AccumuloProps.getInstance(jobContext);
+        String user = MRUtils.AccumuloProps.getUsername(jobContext);
+        AuthenticationToken password = MRUtils.AccumuloProps.getPassword(jobContext);
+        String table = MRUtils.AccumuloProps.getTablename(jobContext);
+        ArgumentChecker.notNull(instance);
+        ArgumentChecker.notNull(table);
+
+        //find the files necessary
+        try {
+            Connector connector = instance.getConnector(user, password);
+            TableOperations tos = connector.tableOperations();
+            String tableId = tos.tableIdMap().get(table);
+            Scanner scanner = connector.createScanner("accumulo.metadata", Authorizations.EMPTY); //TODO: auths?
+            scanner.setRange(new Range(new Text(tableId + "\u0000"), new Text(tableId + "\uFFFD")));
+            scanner.fetchColumnFamily(new Text("file"));
+            List<String> files = new ArrayList<String>();
+            List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+            for (Map.Entry<Key, Value> entry : scanner) {
+                String file = entry.getKey().getColumnQualifier().toString();
+                Path path = new Path(file);
+                FileSystem fs = path.getFileSystem(conf);
+                FileStatus fileStatus = fs.getFileStatus(path);
+                long len = fileStatus.getLen();
+                BlockLocation[] fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, len);
+                files.add(file);
+                fileSplits.add(new FileSplit(path, 0, len, fileBlockLocations[0].getHosts()));
+            }
+            System.out.println(files);
+            return fileSplits;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public RecordReader<Key, Value> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+        return new RecordReader<Key, Value>() {
+
+            private FileSKVIterator fileSKVIterator;
+            private boolean started = false;
+
+            @Override
+            public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+                FileSplit split = (FileSplit) inputSplit;
+                Configuration job = taskAttemptContext.getConfiguration();
+                Path file = split.getPath();
+                FileSystem fs = file.getFileSystem(job);
+                Instance instance = MRUtils.AccumuloProps.getInstance(taskAttemptContext);
+
+                fileSKVIterator = RFileOperations.getInstance().openReader(file.toString(), ALLRANGE,
+                        new HashSet<ByteSequence>(), false, fs, job, instance.getConfiguration());
+            }
+
+            @Override
+            public boolean nextKeyValue() throws IOException, InterruptedException {
+                if (started) {
+                    fileSKVIterator.next();
+                }
+                else {
+                    started = true; // don't move past the first record yet
+                }
+                return fileSKVIterator.hasTop();
+            }
+
+            @Override
+            public Key getCurrentKey() throws IOException, InterruptedException {
+                return fileSKVIterator.getTopKey();
+            }
+
+            @Override
+            public Value getCurrentValue() throws IOException, InterruptedException {
+                return fileSKVIterator.getTopValue();
+            }
+
+            @Override
+            public float getProgress() throws IOException, InterruptedException {
+                return 0;
+            }
+
+            @Override
+            public void close() throws IOException {
+            }
+        };
+    }
+
+    /**
+     * Mapper that has no effect.
+     */
+    @SuppressWarnings("rawtypes")
+    public static class NullMapper extends Mapper {
+        @Override
+        protected void map(Object key, Object value, Context context) throws IOException, InterruptedException {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/MRUtils.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/MRUtils.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/MRUtils.java
new file mode 100644
index 0000000..409c978
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/MRUtils.java
@@ -0,0 +1,317 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.rio.RDFFormat;
+
+import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
+
+/**
+ * Contains constants and static methods for interacting with a
+ * {@link Configuration} and handling options likely to be relevant to Rya
+ * MapReduce jobs. Defines constant property names associated with Accumulo and
+ * Rya options, and some convenience methods to get and set these properties
+ * with respect to a given Configuration.
+ */
+public class MRUtils {
+    /**
+     * Property name for the name of a MapReduce job.
+     */
+    public static final String JOB_NAME_PROP = "mapred.job.name";
+
+    /**
+     * Property name for the Accumulo username.
+     */
+    public static final String AC_USERNAME_PROP = "ac.username";
+    /**
+     * Property name for the Accumulo password.
+     */
+    public static final String AC_PWD_PROP = "ac.pwd";
+
+    /**
+     * Property name for the list of zookeepers.
+     */
+    public static final String AC_ZK_PROP = "ac.zk";
+    /**
+     * Property name for the Accumulo instance name.
+     */
+    public static final String AC_INSTANCE_PROP = "ac.instance";
+    /**
+     * Property name for whether to run against a mock Accumulo instance.
+     */
+    public static final String AC_MOCK_PROP = "ac.mock";
+
+    /**
+     * Property name for TTL; allows using an age-off filter on Accumulo input.
+     */
+    public static final String AC_TTL_PROP = "ac.ttl";
+
+    /**
+     * Property name for scan authorizations when reading data from Accumulo.
+     */
+    public static final String AC_AUTH_PROP = "ac.auth";
+    /**
+     * Property name for default visibility when writing data to Accumulo.
+     */
+    public static final String AC_CV_PROP = "ac.cv";
+
+    /**
+     * Property name for whether to read Accumulo data directly from HDFS
+     * as opposed to through Accumulo itself.
+     */
+    public static final String AC_HDFS_INPUT_PROP = "ac.hdfsinput";
+    /**
+     * Property name for the table layout to use when reading data from Rya.
+     */
+    public static final String TABLE_LAYOUT_PROP = "rdf.tablelayout";
+
+    /**
+     * Property name for the Rya table prefix, identifying the Rya
+     * instance to work with.
+     */
+    public static final String TABLE_PREFIX_PROPERTY = "rdf.tablePrefix";
+    /**
+     * Property name for the RDF serialization format to use, when using RDF
+     * files.
+     */
+    public static final String FORMAT_PROP = "rdf.format";
+    /**
+     * Property name for a file input path, if using file input.
+     */
+    public static final String INPUT_PATH = "input";
+    /**
+     * Property name for specifying a default named graph to use when writing
+     * new statements.
+     */
+    public static final String NAMED_GRAPH_PROP = "rdf.graph";
+
+    public static final String AC_TABLE_PROP = "ac.table";
+    public static final String HADOOP_IO_SORT_MB = "io.sort.mb";
+    public static final ValueFactory vf = new ValueFactoryImpl();
+
+    /**
+     * Gets the TTL from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The TTL that will be applied as an age-off filter for Accumulo
+     *          input data, or null if not set.
+     */
+    public static String getACTtl(Configuration conf) {
+        return conf.get(AC_TTL_PROP);
+    }
+
+    /**
+     * Gets the username from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The configured Accumulo username, or null if not set.
+     */
+    public static String getACUserName(Configuration conf) {
+        return conf.get(AC_USERNAME_PROP);
+    }
+
+    /**
+     * Gets the password from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The configured Accumulo password, or null if not set.
+     */
+    public static String getACPwd(Configuration conf) {
+        return conf.get(AC_PWD_PROP);
+    }
+
+    /**
+     * Gets the zookeepers from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The configured zookeeper list, or null if not set.
+     */
+    public static String getACZK(Configuration conf) {
+        return conf.get(AC_ZK_PROP);
+    }
+
+    /**
+     * Gets the instance name from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The configured Accumulo instance name, or null if not set.
+     */
+    public static String getACInstance(Configuration conf) {
+        return conf.get(AC_INSTANCE_PROP);
+    }
+
+    /**
+     * Gets whether to use a mock instance from a given Configuration.
+     * @param conf          Configuration containing MapReduce tool options.
+     * @param defaultValue  Default choice if the mock property hasn't been
+     *                      explicitly set in the Configuration.
+     * @return  True if a mock instance should be used, false to connect to
+     *          a running Accumulo.
+     */
+    public static boolean getACMock(Configuration conf, boolean defaultValue) {
+        return conf.getBoolean(AC_MOCK_PROP, defaultValue);
+    }
+
+    /**
+     * Gets the table prefix from a given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @return  The configured Rya table prefix, or null if not set.
+     */
+    public static String getTablePrefix(Configuration conf) {
+        return conf.get(TABLE_PREFIX_PROPERTY);
+    }
+
+    /**
+     * Gets the table layout that determines which Rya table to scan for input.
+     * @param   conf            Configuration containing MapReduce tool options.
+     * @param   defaultLayout   The layout to use if the Configuration doesn't
+     *                          specify any layout.
+     * @return  The configured layout to use for reading statements from Rya.
+     */
+    public static TABLE_LAYOUT getTableLayout(Configuration conf, TABLE_LAYOUT defaultLayout) {
+        return TABLE_LAYOUT.valueOf(conf.get(TABLE_LAYOUT_PROP, defaultLayout.toString()));
+    }
+
+    /**
+     * Gets the RDF serialization format to use for parsing RDF files.
+     * @param   conf    Configuration containing MapReduce tool options.
+     * @return  The configured RDFFormat, or null if not set.
+     */
+    public static RDFFormat getRDFFormat(Configuration conf) {
+        return RDFFormat.valueOf(conf.get(FORMAT_PROP));
+    }
+
+    /**
+     * Sets the username in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param str   Accumulo username, used for input and/or output.
+     */
+    public static void setACUserName(Configuration conf, String str) {
+        conf.set(AC_USERNAME_PROP, str);
+    }
+
+    /**
+     * Sets the password in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param str   Accumulo password string, used for input and/or output.
+     */
+    public static void setACPwd(Configuration conf, String str) {
+        conf.set(AC_PWD_PROP, str);
+    }
+
+    /**
+     * Sets the zookeepers in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param str   List of zookeepers to use to connect to Accumulo.
+     */
+    public static void setACZK(Configuration conf, String str) {
+        conf.set(AC_ZK_PROP, str);
+    }
+
+    /**
+     * Sets the instance in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param str   Accumulo instance name, for input and/or output.
+     */
+    public static void setACInstance(Configuration conf, String str) {
+        conf.set(AC_INSTANCE_PROP, str);
+    }
+
+    /**
+     * Sets the TTL in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param str   TTL for Accumulo data. Rows older than this won't be scanned
+     *              as input.
+     */
+    public static void setACTtl(Configuration conf, String str) {
+        conf.set(AC_TTL_PROP, str);
+    }
+
+    /**
+     * Sets whether to connect to a mock Accumulo instance.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param mock  true to use a mock instance, false to attempt to connect
+     *              to a running Accumulo instance.
+     */
+    public static void setACMock(Configuration conf, boolean mock) {
+        conf.setBoolean(AC_MOCK_PROP, mock);
+    }
+
+    /**
+     * Sets the Rya table prefix in the given Configuration.
+     * @param conf      Configuration containing MapReduce tool options.
+     * @param prefix    Prefix of the Rya tables to use for input and/or output.
+     */
+    public static void setTablePrefix(Configuration conf, String prefix) {
+        conf.set(TABLE_PREFIX_PROPERTY, prefix);
+    }
+
+    /**
+     * Sets the table layout in the given Configuration.
+     * @param conf  Configuration containing MapReduce tool options.
+     * @param layout    The Rya core table to scan when using Rya for input.
+     */
+    public static void setTableLayout(Configuration conf, TABLE_LAYOUT layout) {
+        conf.set(TABLE_LAYOUT_PROP, layout.toString());
+    }
+
+    /**
+     * Sets the RDF serialization format in the given Configuration.
+     * @param conf      Configuration containing MapReduce tool options.
+     * @param format    The expected format of any RDF text data.
+     */
+    public static void setRDFFormat(Configuration conf, RDFFormat format) {
+        conf.set(FORMAT_PROP, format.getName());
+    }
+
+    /**
+     * Static class for accessing properties associated with Accumulo input
+     * formats. Can allow input formats that don't extend
+     * {@link InputFormatBase} to still use the same Accumulo input
+     * configuration options.
+     */
+    @SuppressWarnings("rawtypes")
+    public static class AccumuloProps extends InputFormatBase {
+        /**
+         * @throws UnsupportedOperationException always. This class should only be used to access properties.
+         */
+        @Override
+        public RecordReader createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) {
+            throw new UnsupportedOperationException("Accumulo Props just holds properties");
+        }
+        public static Instance getInstance(JobContext  conf) {
+            return InputFormatBase.getInstance(conf);
+        }
+        public static AuthenticationToken getPassword(JobContext  conf) {
+            return InputFormatBase.getAuthenticationToken(conf);
+        }
+        public static String getUsername(JobContext conf) {
+            return InputFormatBase.getPrincipal(conf);
+        }
+        public static String getTablename(JobContext conf) {
+            return InputFormatBase.getInputTableName(conf);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/RdfFileInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/RdfFileInputFormat.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RdfFileInputFormat.java
new file mode 100644
index 0000000..208045e
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RdfFileInputFormat.java
@@ -0,0 +1,443 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.IOException;
+import java.io.PipedReader;
+import java.io.PipedWriter;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.KeyValueLineRecordReader;
+import org.apache.log4j.Logger;
+import org.openrdf.model.Statement;
+import org.openrdf.rio.RDFFormat;
+import org.openrdf.rio.RDFHandler;
+import org.openrdf.rio.RDFHandlerException;
+import org.openrdf.rio.RDFParseException;
+import org.openrdf.rio.RDFParser;
+import org.openrdf.rio.Rio;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.api.resolver.RdfToRyaConversions;
+import mvm.rya.api.resolver.RyaTripleContext;
+
+/**
+ * {@link FileInputFormat} that can read multiple RDF files and convert into
+ * statements.
+ * <p>
+ * Expects all files to use the same RDF serialization format, which must be
+ * provided.
+ * <p>
+ * Reading and parsing is done asynchronously, so entire files need not be
+ * loaded into memory at once. Reading will block when a character buffer is
+ * full, waiting for the parser to consume more data. The parser will block when
+ * a statement buffer is full, waiting for the client to consume the statements
+ * generated so far. This enables large files, particularly useful for N-Triples
+ * and N-Quads formats, which can be parsed one line at a time. The size of each
+ * buffer can be configured. An error will be thrown if the parser takes too
+ * long to respond, and this timeout can be configured.
+ * <p>
+ * Only N-Triples and N-Quads files may be split into multiple
+ * {@link InputSplit}s per file, if large enough. Input is read line-by-line,
+ * and each line of an N-Triples or N-Quads file is self-contained, so any
+ * arbitrary split is valid. This means the number of input splits may be
+ * greater than the number of input files if and only if N-Triples or N-Quads is
+ * given as the RDF serialization format.
+ */
+public class RdfFileInputFormat extends FileInputFormat<LongWritable, RyaStatementWritable> {
+    private static final Logger logger = Logger.getLogger(RdfFileInputFormat.class);
+    private static final String PREFIX = RdfFileInputFormat.class.getSimpleName();
+    private static final String CHAR_BUFFER_SIZE_PROP = PREFIX + ".char.buffer.size";
+    private static final String STATEMENT_BUFFER_SIZE_PROP = PREFIX + ".statement.buffer.size";
+    private static final String TIMEOUT_PROP = PREFIX + ".timeout";
+    private static final String FORMAT_PROP = PREFIX + ".rdf.format";
+
+    private static final RDFFormat DEFAULT_RDF_FORMAT = RDFFormat.RDFXML;
+    private static final int DEFAULT_CHAR_BUFFER_SIZE = 1024*1024;
+    private static final int DEFAULT_STATEMENT_BUFFER_SIZE = 1024;
+    private static final int DEFAULT_TIMEOUT = 20;
+
+    static final RyaStatementWritable DONE = new RyaStatementWritable(null, null); // signals the end of input
+    static final RyaStatementWritable ERROR = new RyaStatementWritable(null, null); // signals some error
+
+    /**
+     * Set the RDF serialization format to parse. All input files must have the
+     * same format.
+     * @param   job     Job to apply the setting to
+     * @param   format  Format of any and all input files
+     */
+    public static void setRDFFormat(Job job, RDFFormat format) {
+        job.getConfiguration().set(FORMAT_PROP, format.getName());
+    }
+
+    /**
+     * Specify the size, in characters, of the input buffer: hold this many
+     * characters in memory before blocking file input.
+     */
+    public static void setCharBufferSize(Job job, int size) {
+        job.getConfiguration().setInt(CHAR_BUFFER_SIZE_PROP, size);
+    }
+
+    /**
+     * Specify the size, in statements, of the parser output buffer: hold this
+     * many Statements in memory before blocking the parser.
+     */
+    public static void setStatementBufferSize(Job job, int size) {
+        job.getConfiguration().setInt(STATEMENT_BUFFER_SIZE_PROP, size);
+    }
+
+    /**
+     * Property to specify the timeout, in seconds:
+     */
+    public static void setTimeout(Job job, int seconds) {
+        job.getConfiguration().setInt(TIMEOUT_PROP, seconds);
+    }
+
+    private RDFFormat getRDFFormat(JobContext context) {
+        String name = context.getConfiguration().get(FORMAT_PROP);
+        return RDFFormat.valueOf(name);
+    }
+
+    /**
+     * Determine whether an input file can be split. If the input format is
+     * configured to be anything other than N-Triples or N-Quads, then the
+     * structure of the file is important and it cannot be split arbitrarily.
+     * Otherwise, default to the superclass logic to determine whether splitting
+     * is appropriate.
+     * @return  true if configured to use a line-based input format and the
+     *          superclass implementation returns true.
+     */
+    @Override
+    protected boolean isSplitable(JobContext context, Path filename) {
+        RDFFormat rdfFormat = getRDFFormat(context);
+        if (RDFFormat.NTRIPLES.equals(rdfFormat) || RDFFormat.NQUADS.equals(rdfFormat)) {
+            return super.isSplitable(context, filename);
+        }
+        return false;
+    }
+
+    /**
+     * Instantiate a RecordReader for a given task attempt.
+     * @param   inputSplit  Input split to handle, may refer to part or all of
+     *                      an RDF file
+     * @param   taskAttemptContext  Contains configuration options.
+     * @return  A RecordReader that reads and parses RDF text.
+     */
+    @Override
+    public RecordReader<LongWritable, RyaStatementWritable> createRecordReader(InputSplit inputSplit,
+            TaskAttemptContext taskAttemptContext) {
+        Configuration conf = taskAttemptContext.getConfiguration();
+        RDFFormat format = getRDFFormat(taskAttemptContext);
+        if (format == null) {
+            format = DEFAULT_RDF_FORMAT;
+        }
+        int charBufferSize = conf.getInt(CHAR_BUFFER_SIZE_PROP, DEFAULT_CHAR_BUFFER_SIZE);
+        int statementBufferSize = conf.getInt(STATEMENT_BUFFER_SIZE_PROP, DEFAULT_STATEMENT_BUFFER_SIZE);
+        int timeoutSeconds = conf.getInt(TIMEOUT_PROP, DEFAULT_TIMEOUT);
+        return new RdfFileRecordReader(format, charBufferSize, statementBufferSize, timeoutSeconds);
+    }
+
+    /**
+     * Reads RDF files and generates RyaStatementWritables. Reads and parses
+     * data in parallel, so the entire file need not be loaded at once.
+     */
+    class RdfFileRecordReader extends RecordReader<LongWritable, RyaStatementWritable> implements RDFHandler {
+        private RecordReader<Text, Text> lineReader;
+        private final PipedWriter pipeOut;
+        private final PipedReader pipeIn;
+        private final RDFParser rdfParser;
+        final BlockingQueue<RyaStatementWritable> statementCache;
+
+        private long lineCount = 0;
+        private long statementCount = 0;
+        private RyaTripleContext tripleContext;
+        private RyaStatementWritable nextStatement = null;
+        private int timeoutSeconds;
+        private boolean noMoreStatements = false;
+
+        Thread readerThread;
+        Thread parserThread;
+        private Exception threadException;
+
+        /**
+         * Instantiates the RecordReader.
+         * @param format    RDF serialization format to parse.
+         * @param charBufferSize    Number of input characters to hold in
+         *                          memory; if exceeded, wait until the parser
+         *                          thread consumes some text before proceeding
+         *                          with reading input.
+         * @param statementBufferSize   Number of output statements to hold in
+         *                              memory; if exceeded, wait until the
+         *                              client consumes data before proceeding
+         *                              with parsing.
+         * @param timeoutSeconds    Number of seconds to wait for the parser
+         *                          thread to provide the next statement (or
+         *                          state that there are none). If exceeded,
+         *                          abort.
+         */
+        RdfFileRecordReader(RDFFormat format, int charBufferSize, int statementBufferSize, int timeoutSeconds) {
+            rdfParser = Rio.createParser(format);
+            rdfParser.setRDFHandler(this);
+            statementCache = new LinkedBlockingQueue<RyaStatementWritable>(statementBufferSize);
+            pipeOut = new PipedWriter();
+            pipeIn = new PipedReader(charBufferSize);
+            this.timeoutSeconds = timeoutSeconds;
+            logger.info("Initializing RecordReader with parameters:");
+            logger.info("\tRDF serialization format = " + format.getName());
+            logger.info("\tinput buffer size = " + charBufferSize + " characters");
+            logger.info("\tstatement cache size = " + statementBufferSize);
+            logger.info("\tparser timeout = " + timeoutSeconds + " seconds");
+        }
+
+        /**
+         * Starts up one thread for reading text data (via an internal line
+         * based RecordReader) and one thread for receiving and parsing that
+         * data, each blocking when their respective buffers are full.
+         * @param   inputSplit          The section of data to read
+         * @param   taskAttemptContext  Contains configuration variables
+         * @throws  IOException if an error is encountered initializing the line
+         *          RecordReader or piping its output to the parser thread.
+         * @throws  InterruptedException if an error is encountered initializing
+         *          the line RecordReader
+         */
+        @Override
+        public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+            Configuration conf = taskAttemptContext.getConfiguration();
+            lineReader = new KeyValueLineRecordReader(conf);
+            lineReader.initialize(inputSplit, taskAttemptContext);
+            tripleContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
+            pipeIn.connect(pipeOut);
+
+            readerThread = new Thread(Thread.currentThread().getName() + " -- reader thread") {
+                @Override
+                public void run() {
+                    try {
+                        logger.info("Starting file reader");
+                        while (lineReader.nextKeyValue()) {
+                            Text key = lineReader.getCurrentKey();
+                            Text value = lineReader.getCurrentValue();
+                            pipeOut.write(key.toString());
+                            if (value.getLength() > 0) {
+                                pipeOut.write(KeyValueLineRecordReader.KEY_VALUE_SEPERATOR);
+                                pipeOut.write(value.toString());
+                            }
+                            pipeOut.write('\n');
+                            lineCount++;
+                        }
+                        logger.info("Reached end of input text; read " + lineCount + " lines in total");
+                    } catch (IOException | InterruptedException e) {
+                        logger.error("Error processing line " + (lineCount+1) + " of input", e);
+                        fail(e, this);
+                        throw new RuntimeException(e.getMessage(), e);
+                    }
+                    finally {
+                        try { lineReader.close(); } catch (IOException e) { logger.warn(e); }
+                        try { pipeOut.close(); } catch (IOException e) { logger.warn(e); }
+                    }
+                }
+            };
+
+            parserThread = new Thread(Thread.currentThread().getName() + " -- parser thread") {
+                @Override
+                public void run() {
+                    try {
+                        logger.info("Starting parser");
+                        rdfParser.parse(pipeIn, "");
+                    }
+                    catch (RDFHandlerException | RDFParseException | IOException e) {
+                        logger.error(e.getMessage(), e);
+                        fail(e, this);
+                        throw new RuntimeException(e.getMessage(), e);
+                    }
+                    finally {
+                        try { pipeIn.close(); } catch (IOException e) { logger.warn(e); }
+                    }
+                }
+            };
+            readerThread.start();
+            parserThread.start();
+        }
+
+        private void fail(Exception e, Thread source) {
+            // Notify the main RecordReader of the error
+            statementCache.offer(ERROR);
+            threadException = e;
+            // Kill the reader thread if necessary
+            if (source != readerThread && readerThread.isAlive()) {
+                readerThread.interrupt();
+            }
+            // Kill the parser thread if necessary
+            if (source != parserThread && parserThread.isAlive()) {
+                parserThread.interrupt();
+            }
+        }
+
+        /**
+         * Loads the next statement, if there is one, and returns whether there
+         * is one. Receives statements from the parser thread via a blocking
+         * queue.
+         * @throws  InterruptedException if interrupted while waiting for a
+         *          statement to show up in the queue.
+         * @throws  IOException if the parser thread doesn't respond after the
+         *          configured timeout, or if any thread reports an error.
+         * @return  true if a valid statement was loaded, or false if there are
+         *          no more statements in this input split.
+         */
+        @Override
+        public boolean nextKeyValue() throws IOException, InterruptedException {
+            if (noMoreStatements) {
+                return false;
+            }
+            nextStatement = statementCache.poll(timeoutSeconds, TimeUnit.SECONDS);
+            if (nextStatement == null) {
+                    throw new IOException("Parser neither sending results nor signaling end of data after "
+                        + timeoutSeconds + " seconds.");
+            }
+            else if (nextStatement == DONE) {
+                logger.info("Reached end of parsed RDF; read " +  statementCount + " statements in total.");
+                nextStatement = null;
+                noMoreStatements = true;
+                return false;
+            }
+            else if (nextStatement == ERROR) {
+                nextStatement = null;
+                noMoreStatements = true;
+                throw new IOException("Error detected processing input.", threadException);
+            }
+            statementCount++;
+            return true;
+        }
+
+        /**
+         * Gets the current key.
+         * @return  the number of statements read so far, or null if all input
+         *          has been read.
+         */
+        @Override
+        public LongWritable getCurrentKey() {
+            if (noMoreStatements) {
+                return null;
+            }
+            return new LongWritable(statementCount);
+        }
+
+        /**
+         * Gets the current value.
+         * @return  a RyaStatementWritable loaded from RDF data, or null if all
+         *          input has been read.
+         */
+        @Override
+        public RyaStatementWritable getCurrentValue() {
+            return nextStatement;
+        }
+
+        /**
+         * Gets the progress of the underlying line-based Record Reader. Does
+         * not include any information about the progress of the parser.
+         * @return  The proportion of text input that has been read.
+         * @throws  IOException if thrown by the internal RecordReader.
+         * @throws  InterruptedException if thrown by the internal RecordReader.
+         */
+        @Override
+        public float getProgress() throws IOException, InterruptedException {
+            return lineReader.getProgress();
+        }
+
+        /**
+         * Closes all the underlying resources.
+         */
+        @Override
+        public void close() {
+            if (parserThread.isAlive()) {
+                parserThread.interrupt();
+            }
+            if (readerThread.isAlive()) {
+                readerThread.interrupt();
+            }
+            try { lineReader.close(); } catch (IOException e) { logger.warn(e); }
+            try { pipeOut.close(); } catch (IOException e) { logger.warn(e); }
+            try { pipeIn.close(); } catch (IOException e) { logger.warn(e); }
+        }
+
+        /**
+         * Has no effect.
+         */
+        @Override
+        public void startRDF() throws RDFHandlerException {
+        }
+
+        /**
+         * Add a dummy item to the queue to signal that there will be no more
+         * statements.
+         * @throws  RDFHandlerException     if interrupted while waiting for
+         *          the blocking queue to be ready to accept the done signal.
+         */
+        @Override
+        public void endRDF() throws RDFHandlerException {
+            logger.info("Finished parsing RDF");
+            try {
+                statementCache.put(DONE);
+            } catch (InterruptedException e) {
+                throw new RDFHandlerException("Interrupted while waiting to add done signal to statement queue", e);
+            }
+        }
+
+        /**
+         * Has no effect.
+         */
+        @Override
+        public void handleNamespace(String s, String s1) throws RDFHandlerException {
+        }
+
+        /**
+         * Convert the {@link Statement} to a {@link RyaStatement}, wrap it in a
+         * {@link RyaStatementWritable}, and add it to the queue.
+         * @throws  RDFHandlerException     if interrupted while waiting for the
+         *          blocking queue to be ready to accept statement data.
+         */
+        @Override
+        public void handleStatement(Statement statement) throws RDFHandlerException {
+            try {
+                statementCache.put(new RyaStatementWritable(RdfToRyaConversions.convertStatement(statement), tripleContext));
+            } catch (InterruptedException e) {
+                throw new RDFHandlerException("Interrupted while waiting to add parsed statement to the statement queue", e);
+            }
+        }
+
+        /**
+         * Has no effect.
+         */
+        @Override
+        public void handleComment(String s) {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaInputFormat.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaInputFormat.java
new file mode 100644
index 0000000..b6545b6
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaInputFormat.java
@@ -0,0 +1,130 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.mapreduce.AbstractInputFormat;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.resolver.RyaTripleContext;
+import mvm.rya.api.resolver.triple.TripleRow;
+import mvm.rya.api.resolver.triple.TripleRowResolverException;
+
+/**
+ * Subclass of {@link AbstractInputFormat} for reading
+ * {@link RyaStatementWritable}s directly from a running Rya instance.
+ */
+public class RyaInputFormat extends AbstractInputFormat<Text, RyaStatementWritable> {
+    /**
+     * Instantiates a RecordReader for this InputFormat and a given task and
+     * input split.
+     * @param   split   Defines the portion of the input this RecordReader is
+     *                  responsible for.
+     * @param   context     The context of the task.
+     * @return A RecordReader that can be used to fetch RyaStatementWritables.
+     */
+    @Override
+    public RecordReader<Text, RyaStatementWritable> createRecordReader(InputSplit split, TaskAttemptContext context) {
+        return new RyaStatementRecordReader();
+    }
+
+    /**
+     * Sets the table layout to use.
+     * @param conf  Configuration to set the layout in.
+     * @param layout    Statements will be read from the Rya table associated
+     *                  with this layout.
+     */
+    public static void setTableLayout(Job conf, TABLE_LAYOUT layout) {
+        conf.getConfiguration().set(MRUtils.TABLE_LAYOUT_PROP, layout.name());
+    }
+
+    /**
+     * Retrieves RyaStatementWritable objects from Accumulo tables.
+     */
+    public class RyaStatementRecordReader extends AbstractRecordReader<Text, RyaStatementWritable> {
+        private RyaTripleContext ryaContext;
+        private TABLE_LAYOUT tableLayout;
+
+        @Override
+        protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName,
+                RangeInputSplit split) {
+        }
+
+        /**
+         * Initializes the RecordReader.
+         * @param   inSplit Defines the portion of data to read.
+         * @param   attempt Context for this task attempt.
+         * @throws IOException if thrown by the superclass's initialize method.
+         */
+        @Override
+        public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
+            super.initialize(inSplit, attempt);
+            this.tableLayout = MRUtils.getTableLayout(attempt.getConfiguration(), TABLE_LAYOUT.OSP);
+            //TODO verify that this is correct
+            this.ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(attempt.getConfiguration()));
+        }
+
+        /**
+         * Load the next statement by converting the next Accumulo row to a
+         * statement, and make the new (key,value) pair available for retrieval.
+         * @return true if another (key,value) pair was fetched and is ready to
+         *          be retrieved, false if there was none.
+         * @throws  IOException if a row was loaded but could not be converted
+         *          to a statement.
+         */
+        @Override
+        public boolean nextKeyValue() throws IOException {
+            if (!scannerIterator.hasNext())
+                return false;
+            Entry<Key, Value> entry = scannerIterator.next();
+            ++numKeysRead;
+            currentKey = entry.getKey();
+            try {
+                currentK = currentKey.getRow();
+                RyaStatement stmt = this.ryaContext.deserializeTriple(this.tableLayout,
+                        new TripleRow(entry.getKey().getRow().getBytes(),
+                                entry.getKey().getColumnFamily().getBytes(),
+                                entry.getKey().getColumnQualifier().getBytes(),
+                                entry.getKey().getTimestamp(),
+                                entry.getKey().getColumnVisibility().getBytes(),
+                                entry.getValue().get()));
+                RyaStatementWritable writable = new RyaStatementWritable();
+                writable.setRyaStatement(stmt);
+                currentV = writable;
+            } catch(TripleRowResolverException e) {
+                throw new IOException(e);
+            }
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
new file mode 100644
index 0000000..c9f7ffe
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
@@ -0,0 +1,597 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MultiTableBatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.log4j.Logger;
+import org.openrdf.model.Statement;
+import org.openrdf.model.vocabulary.XMLSchema;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRdfConstants;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaType;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.api.resolver.RdfToRyaConversions;
+import mvm.rya.api.resolver.RyaTripleContext;
+import mvm.rya.indexing.FreeTextIndexer;
+import mvm.rya.indexing.GeoIndexer;
+import mvm.rya.indexing.TemporalIndexer;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.accumulo.entity.EntityCentricIndex;
+import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
+import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
+import mvm.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
+
+/**
+ * {@link OutputFormat} that uses Rya, the {@link GeoIndexer}, the
+ * {@link FreeTextIndexer}, the {@link TemporalIndexer}, and the
+ * {@link EntityCentricIndex} as the sink of triple data. This
+ * OutputFormat ignores the Keys and only writes the Values to Rya.
+ * <p>
+ * The user must specify connection parameters for Rya, {@link GeoIndexer},
+ * {@link FreeTextIndexer}, {@link TemporalIndexer}, and
+ * {@link EntityCentricIndex}, if secondary indexing is desired.
+ */
+public class RyaOutputFormat extends OutputFormat<Writable, RyaStatementWritable> {
+    private static final Logger logger = Logger.getLogger(RyaOutputFormat.class);
+
+    private static final String PREFIX = RyaOutputFormat.class.getSimpleName();
+    private static final String MAX_MUTATION_BUFFER_SIZE = PREFIX + ".maxmemory";
+    private static final String ENABLE_FREETEXT = PREFIX + ".freetext.enable";
+    private static final String ENABLE_GEO = PREFIX + ".geo.enable";
+    private static final String ENABLE_TEMPORAL = PREFIX + ".temporal.enable";
+    private static final String ENABLE_ENTITY = PREFIX + ".entity.enable";
+    private static final String ENABLE_CORE = PREFIX + ".coretables.enable";
+    private static final String OUTPUT_PREFIX_PROPERTY = PREFIX + ".tablePrefix";
+    private static final String CV_PROPERTY = PREFIX + ".cv.default";
+    private static final String CONTEXT_PROPERTY = PREFIX + ".context";
+
+    /**
+     * Set the default visibility of output: any statement whose visibility is
+     * null will be written with this visibility instead. If not specified, use
+     * an empty authorizations list.
+     * @param job Job to apply the setting to.
+     * @param visibility A comma-separated list of authorizations.
+     */
+    public static void setDefaultVisibility(Job job, String visibility) {
+        if (visibility != null) {
+            job.getConfiguration().set(CV_PROPERTY, visibility);
+        }
+    }
+
+    /**
+     * Set the default context (named graph) for any output: any statement whose
+     * context is null will be written with this context instead. If not
+     * specified, don't write any context.
+     * @param job Job to apply the setting to.
+     * @param context A context string, should be a syntactically valid URI.
+     */
+    public static void setDefaultContext(Job job, String context) {
+        if (context != null) {
+            job.getConfiguration().set(CONTEXT_PROPERTY, context);
+        }
+    }
+
+    /**
+     * Set the table prefix for output.
+     * @param job Job to apply the setting to.
+     * @param prefix The common prefix to all rya tables that output will be written to.
+     */
+    public static void setTablePrefix(Job job, String prefix) {
+        job.getConfiguration().set(OUTPUT_PREFIX_PROPERTY, prefix);
+    }
+
+    /**
+     * Set whether the free text index is enabled. Defaults to true.
+     * @param job Job to apply the setting to.
+     * @param enable Whether this job should add its output statements to the free text index.
+     */
+    public static void setFreeTextEnabled(Job job, boolean enable) {
+        job.getConfiguration().setBoolean(ENABLE_FREETEXT, enable);
+    }
+
+    /**
+     * Set whether the geo index is enabled. Defaults to true.
+     * @param job Job to apply the setting to.
+     * @param enable Whether this job should add its output statements to the geo index.
+     */
+    public static void setGeoEnabled(Job job, boolean enable) {
+        job.getConfiguration().setBoolean(ENABLE_GEO, enable);
+    }
+
+    /**
+     * Set whether the temporal index is enabled. Defaults to true.
+     * @param job Job to apply the setting to.
+     * @param enable Whether this job should add its output statements to the temporal index.
+     */
+    public static void setTemporalEnabled(Job job, boolean enable) {
+        job.getConfiguration().setBoolean(ENABLE_TEMPORAL, enable);
+    }
+
+    /**
+     * Set whether the entity-centric index is enabled. Defaults to true.
+     * @param job Job to apply the setting to.
+     * @param enable Whether this job should add its output statements to the entity-centric index.
+     */
+    public static void setEntityEnabled(Job job, boolean enable) {
+        job.getConfiguration().setBoolean(ENABLE_ENTITY, enable);
+    }
+
+    /**
+     * Set whether to insert to the core Rya tables (spo, osp, po). Defaults to true.
+     * @param job Job to apply the setting to.
+     * @param enable Whether this job should output to the core tables.
+     */
+    public static void setCoreTablesEnabled(Job job, boolean enable) {
+        job.getConfiguration().setBoolean(ENABLE_CORE, enable);
+    }
+
+    /**
+     * Configure a job to use a mock Accumulo instance.
+     * @param job Job to configure
+     * @param instance Name of the mock instance
+     */
+    public static void setMockInstance(Job job, String instance) {
+        AccumuloOutputFormat.setMockInstance(job, instance);
+        job.getConfiguration().setBoolean(ConfigUtils.USE_MOCK_INSTANCE, true);
+        job.getConfiguration().setBoolean(MRUtils.AC_MOCK_PROP, true);
+    }
+
+    /**
+     * Verify that all of the enabled indexers can be initialized.
+     * @param   jobContext  Context containing configuration
+     * @throws  IOException if initializing the core Rya indexer fails.
+     */
+    @Override
+    public void checkOutputSpecs(JobContext jobContext) throws IOException {
+        Configuration conf = jobContext.getConfiguration();
+        // make sure that all of the indexers can connect
+        getGeoIndexer(conf);
+        getFreeTextIndexer(conf);
+        getTemporalIndexer(conf);
+        getRyaIndexer(conf);
+    }
+
+    /**
+     * Get the OutputCommitter for this OutputFormat.
+     * @param   context Context of the MapReduce task
+     * @return  A committer whose method implementations are empty.
+     */
+    @Override
+    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException {
+        // copied from AccumuloOutputFormat
+        return new NullOutputFormat<Text, Mutation>().getOutputCommitter(context);
+    }
+
+    /**
+     * Get this OutputFormat's RecordWriter.
+     * @param   context     Context of the MapReduce task
+     * @return  A RecordWriter that writes statements to Rya tables.
+     * @throws  IOException if any enabled indexers can't be initialized
+     */
+    @Override
+    public RecordWriter<Writable, RyaStatementWritable> getRecordWriter(TaskAttemptContext context) throws IOException {
+        return new RyaRecordWriter(context);
+    }
+
+    private static GeoIndexer getGeoIndexer(Configuration conf) {
+        if (!conf.getBoolean(ENABLE_GEO, true)) {
+            return null;
+        }
+        GeoMesaGeoIndexer geo = new GeoMesaGeoIndexer();
+        geo.setConf(conf);
+        return geo;
+    }
+
+    private static FreeTextIndexer getFreeTextIndexer(Configuration conf) {
+        if (!conf.getBoolean(ENABLE_FREETEXT, true)) {
+            return null;
+        }
+        AccumuloFreeTextIndexer freeText = new AccumuloFreeTextIndexer();
+        freeText.setConf(conf);
+        return freeText;
+    }
+
+    private static TemporalIndexer getTemporalIndexer(Configuration conf) {
+        if (!conf.getBoolean(ENABLE_TEMPORAL, true)) {
+            return null;
+        }
+        AccumuloTemporalIndexer temporal = new AccumuloTemporalIndexer();
+        temporal.setConf(conf);
+        return temporal;
+    }
+
+    private static EntityCentricIndex getEntityIndexer(Configuration conf) {
+        if (!conf.getBoolean(ENABLE_ENTITY, true)) {
+            return null;
+        }
+        EntityCentricIndex entity = new EntityCentricIndex();
+        entity.setConf(conf);
+        return entity;
+    }
+
+    private static AccumuloRyaDAO getRyaIndexer(Configuration conf) throws IOException {
+        try {
+            if (!conf.getBoolean(ENABLE_CORE, true)) {
+                return null;
+            }
+            AccumuloRyaDAO ryaIndexer = new AccumuloRyaDAO();
+            Connector conn = ConfigUtils.getConnector(conf);
+            ryaIndexer.setConnector(conn);
+
+            AccumuloRdfConfiguration ryaConf = new AccumuloRdfConfiguration();
+
+            String tablePrefix = conf.get(OUTPUT_PREFIX_PROPERTY, null);
+            if (tablePrefix != null) {
+                ryaConf.setTablePrefix(tablePrefix);
+            }
+            ryaConf.setDisplayQueryPlan(false);
+            ryaIndexer.setConf(ryaConf);
+            ryaIndexer.init();
+            return ryaIndexer;
+        } catch (AccumuloException e) {
+            logger.error("Cannot create RyaIndexer", e);
+            throw new IOException(e);
+        } catch (AccumuloSecurityException e) {
+            logger.error("Cannot create RyaIndexer", e);
+            throw new IOException(e);
+        } catch (RyaDAOException e) {
+            logger.error("Cannot create RyaIndexer", e);
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * RecordWriter that takes in {@link RyaStatementWritable}s and writes them
+     * to Rya tables.
+     */
+    public static class RyaRecordWriter extends RecordWriter<Writable, RyaStatementWritable>
+            implements Closeable, Flushable {
+        private static final Logger logger = Logger.getLogger(RyaRecordWriter.class);
+
+        private FreeTextIndexer freeTextIndexer;
+        private GeoIndexer geoIndexer;
+        private TemporalIndexer temporalIndexer;
+        private EntityCentricIndex entityIndexer;
+        private AccumuloRyaDAO ryaIndexer;
+        private RyaTripleContext tripleContext;
+        private MultiTableBatchWriter writer;
+        private byte[] cv = AccumuloRdfConstants.EMPTY_CV.getExpression();
+        private RyaURI defaultContext = null;
+
+        private static final long ONE_MEGABYTE = 1024L * 1024L;
+        private static final long AVE_STATEMENT_SIZE = 100L;
+
+        private long bufferSizeLimit;
+        private long bufferCurrentSize = 0;
+
+        private ArrayList<RyaStatement> buffer;
+
+        /**
+         * Constructor.
+         * @param context Context for MapReduce task
+         * @throws  IOException if the core Rya indexer or entity indexer can't
+         *          be initialized
+         */
+        public RyaRecordWriter(TaskAttemptContext context) throws IOException {
+            this(context.getConfiguration());
+        }
+
+        /**
+         * Constructor.
+         * @param conf Configuration containing any relevant options.
+         * @throws  IOException if the core Rya indexer or entity indexer can't
+         *          be initialized
+         */
+        public RyaRecordWriter(Configuration conf) throws IOException {
+            // set the visibility
+            String visibility = conf.get(CV_PROPERTY);
+            if (visibility != null) {
+                cv = visibility.getBytes();
+            }
+            // set the default context
+            String context = conf.get(CONTEXT_PROPERTY, "");
+            if (context != null && !context.isEmpty()) {
+                defaultContext = new RyaURI(context);
+            }
+
+            // set up the buffer
+            bufferSizeLimit = conf.getLong(MAX_MUTATION_BUFFER_SIZE, ONE_MEGABYTE);
+            int bufferCapacity = (int) (bufferSizeLimit / AVE_STATEMENT_SIZE);
+            buffer = new ArrayList<RyaStatement>(bufferCapacity);
+
+            // set up the indexers
+            freeTextIndexer = getFreeTextIndexer(conf);
+            geoIndexer = getGeoIndexer(conf);
+            temporalIndexer = getTemporalIndexer(conf);
+            entityIndexer = getEntityIndexer(conf);
+            ryaIndexer = getRyaIndexer(conf);
+
+            // The entity index needs a batch writer -- typically it uses the DAO's, but decoupling
+            // them lets it be used with or without the core tables, like the other indexers.
+            if (entityIndexer != null) {
+                Connector conn;
+                try {
+                    conn = ConfigUtils.getConnector(conf);
+                } catch (AccumuloException | AccumuloSecurityException e) {
+                    throw new IOException("Error connecting to Accumulo for entity index output", e);
+                }
+                BatchWriterConfig batchWriterConfig = new BatchWriterConfig();
+                batchWriterConfig.setMaxMemory(RdfCloudTripleStoreConstants.MAX_MEMORY);
+                batchWriterConfig.setTimeout(RdfCloudTripleStoreConstants.MAX_TIME, TimeUnit.MILLISECONDS);
+                batchWriterConfig.setMaxWriteThreads(RdfCloudTripleStoreConstants.NUM_THREADS);
+                writer = conn.createMultiTableBatchWriter(batchWriterConfig);
+                entityIndexer.setMultiTableBatchWriter(writer);
+            }
+
+            // update fields used for metrics
+            startTime = System.currentTimeMillis();
+            lastCommitFinishTime = startTime;
+
+            // set up the triple context
+            tripleContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
+        }
+
+        /**
+         * Write any buffered statements to Accumulo.
+         * @throws IOException if any indexer can't be flushed.
+         */
+        @Override
+        public void flush() throws IOException {
+            flushBuffer();
+        }
+
+        /**
+         * Close all indexers.
+         */
+        @Override
+        public void close() {
+            close(null);
+        }
+
+        /**
+         * Close all indexers.
+         * @param   paramTaskAttemptContext     Unused.
+         */
+        @Override
+        public void close(TaskAttemptContext paramTaskAttemptContext) {
+            // close everything. log errors
+            try {
+                flush();
+            } catch (IOException e) {
+                logger.error("Error flushing the buffer on RyaOutputFormat Close", e);
+            }
+            try {
+                if (geoIndexer != null)
+                    geoIndexer.close();
+            } catch (IOException e) {
+                logger.error("Error closing the geoIndexer on RyaOutputFormat Close", e);
+            }
+            try {
+                if (freeTextIndexer != null)
+                    freeTextIndexer.close();
+            } catch (IOException e) {
+                logger.error("Error closing the freetextIndexer on RyaOutputFormat Close", e);
+            }
+            try {
+                if (temporalIndexer != null)
+                    temporalIndexer.close();
+            } catch (IOException e) {
+                logger.error("Error closing the temporalIndexer on RyaOutputFormat Close", e);
+            }
+            try {
+                if (entityIndexer != null)
+                    entityIndexer.close();
+            } catch (IOException e) {
+                logger.error("Error closing the entityIndexer on RyaOutputFormat Close", e);
+            }
+            try {
+                if (ryaIndexer != null)
+                    ryaIndexer.destroy();
+            } catch (RyaDAOException e) {
+                logger.error("Error closing RyaDAO on RyaOutputFormat Close", e);
+            }
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (MutationsRejectedException e) {
+                    logger.error("Error closing MultiTableBatchWriter on RyaOutputFormat Close", e);
+                }
+            }
+        }
+
+        /**
+         * Write a {@link Statement} to Rya. Adds the statement to a buffer, and
+         * flushes the statement buffer to the database if full.
+         * @param   statement   Statement to insert to Rya.
+         * @throws  IOException if writing to Accumulo fails.
+         */
+        public void write(Statement statement) throws IOException {
+            write(RdfToRyaConversions.convertStatement(statement));
+        }
+
+        /**
+         * Writes a RyaStatement to Rya. Adds the statement to a buffer, and
+         * flushes the statement buffer to the database if full.
+         * @param   ryaStatement   Statement to insert to Rya.
+         * @throws  IOException if writing to Accumulo fails.
+         */
+        public void write(RyaStatement ryaStatement) throws IOException {
+            write(NullWritable.get(), new RyaStatementWritable(ryaStatement, tripleContext));
+        }
+
+        /**
+         * Writes a (key,value) pair to Rya. Adds the statement to a buffer, and
+         * flushes the statement buffer to the database if full.
+         * @param   key     Arbitrary Writable, not used.
+         * @param   value   Contains statement to insert to Rya.
+         * @throws  IOException if writing to Accumulo fails.
+         */
+        @Override
+        public void write(Writable key, RyaStatementWritable value) throws IOException {
+            RyaStatement ryaStatement = value.getRyaStatement();
+            if (ryaStatement.getColumnVisibility() == null) {
+                ryaStatement.setColumnVisibility(cv);
+            }
+            if (ryaStatement.getContext() == null) {
+                ryaStatement.setContext(defaultContext);
+            }
+            buffer.add(ryaStatement);
+            bufferCurrentSize += statementSize(ryaStatement);
+            if (bufferCurrentSize >= bufferSizeLimit) {
+                flushBuffer();
+            }
+        }
+
+        private int statementSize(RyaStatement ryaStatement) {
+            RyaURI subject = ryaStatement.getSubject();
+            RyaURI predicate = ryaStatement.getPredicate();
+            RyaType object = ryaStatement.getObject();
+            RyaURI context = ryaStatement.getContext();
+            int size = 3 + subject.getData().length() + predicate.getData().length() + object.getData().length();
+            if (!XMLSchema.ANYURI.equals(object.getDataType())) {
+                size += 2 + object.getDataType().toString().length();
+            }
+            if (context != null) {
+                size += context.getData().length();
+            }
+            return size;
+        }
+
+        // fields for storing metrics
+        private long startTime = 0;
+        private long lastCommitFinishTime = 0;
+        private long totalCommitRecords = 0;
+
+        private double totalReadDuration = 0;
+        private double totalWriteDuration = 0;
+
+        private long commitCount = 0;
+
+        private void flushBuffer() throws IOException {
+            totalCommitRecords += buffer.size();
+            commitCount++;
+
+            long startCommitTime = System.currentTimeMillis();
+
+            logger.info(String.format("(C-%d) Flushing buffer with %,d objects and %,d bytes", commitCount, buffer.size(),
+                    bufferCurrentSize));
+
+            double readingDuration = (startCommitTime - lastCommitFinishTime) / 1000.;
+            totalReadDuration += readingDuration;
+            double currentReadRate = buffer.size() / readingDuration;
+            double totalReadRate = totalCommitRecords / totalReadDuration;
+
+            // Print "reading" metrics
+            logger.info(String.format("(C-%d) (Reading) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, readingDuration,
+                    currentReadRate, totalReadRate));
+
+            // write to geo
+            if (geoIndexer != null) {
+                geoIndexer.storeStatements(buffer);
+                geoIndexer.flush();
+            }
+
+            // write to free text
+            if (freeTextIndexer != null) {
+                freeTextIndexer.storeStatements(buffer);
+                freeTextIndexer.flush();
+            }
+
+            // write to temporal
+            if (temporalIndexer != null) {
+                temporalIndexer.storeStatements(buffer);
+                temporalIndexer.flush();
+            }
+
+            // write to entity
+            if (entityIndexer != null && writer != null) {
+                entityIndexer.storeStatements(buffer);
+                try {
+                    writer.flush();
+                } catch (MutationsRejectedException e) {
+                    throw new IOException("Error flushing data to Accumulo for entity indexing", e);
+                }
+            }
+
+            // write to rya
+            try {
+                if (ryaIndexer != null) {
+                    ryaIndexer.add(buffer.iterator());
+                }
+            } catch (RyaDAOException e) {
+                logger.error("Cannot write statement to Rya", e);
+                throw new IOException(e);
+            }
+
+            lastCommitFinishTime = System.currentTimeMillis();
+
+            double writingDuration = (lastCommitFinishTime - startCommitTime) / 1000.;
+            totalWriteDuration += writingDuration;
+            double currentWriteRate = buffer.size() / writingDuration;
+            double totalWriteRate = totalCommitRecords / totalWriteDuration;
+
+            // Print "writing" stats
+            logger.info(String.format("(C-%d) (Writing) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, writingDuration,
+                    currentWriteRate, totalWriteRate));
+
+            double processDuration = writingDuration + readingDuration;
+            double totalProcessDuration = totalWriteDuration + totalReadDuration;
+            double currentProcessRate = buffer.size() / processDuration;
+            double totalProcessRate = totalCommitRecords / (totalProcessDuration);
+
+            // Print "total" stats
+            logger.info(String.format("(C-%d) (Total) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, processDuration,
+                    currentProcessRate, totalProcessRate));
+
+            // clear the buffer
+            buffer.clear();
+            bufferCurrentSize = 0L;
+        }
+    }
+}


[2/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Posted by pu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
new file mode 100644
index 0000000..cdc3235
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/RyaStatementWritable.java
@@ -0,0 +1,256 @@
+package mvm.rya.accumulo.mr;
+
+/*
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.lang.builder.CompareToBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparable;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.resolver.RyaTripleContext;
+import mvm.rya.api.resolver.triple.TripleRow;
+import mvm.rya.api.resolver.triple.TripleRowResolverException;
+
+/**
+ * Basic {@link WritableComparable} for using Rya data with Hadoop.
+ * RyaStatementWritable wraps a {@link RyaStatement}, which in turn represents a
+ * statement as  a collection of {@link mvm.rya.api.domain.RyaURI} and
+ * {@link mvm.rya.api.domain.RyaType} objects.
+ * <p>
+ * This class is mutable, like all {@link org.apache.hadoop.io.Writable}s. When
+ * used as Mapper or Reducer input, the Hadoop framework will typically reuse
+ * the same object to load the next record. However, loading the next record
+ * will create a new RyaStatement internally. Therefore, if a statement must be
+ * stored for any length of time, be sure to extract the internal RyaStatement.
+ */
+public class RyaStatementWritable implements WritableComparable<RyaStatementWritable> {
+    private RyaTripleContext ryaContext;
+    private RyaStatement ryaStatement;
+
+    /**
+     * Instantiates a RyaStatementWritable with the default RyaTripleContext.
+     * @param conf  Unused.
+     */
+    public RyaStatementWritable(Configuration conf) {
+        this();
+    }
+    /**
+     * Instantiates a RyaStatementWritable with a given context.
+     * @param ryaContext    Context used for reading and writing the statement.
+     */
+    public RyaStatementWritable(RyaTripleContext ryaContext) {
+        this.ryaContext = ryaContext;
+    }
+    /**
+     * Instantiates a RyaStatementWritable with the default RyaTripleContext.
+     */
+    public RyaStatementWritable() {
+        this.ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration());
+    }
+    /**
+     * Instantiates a RyaStatementWritable with a given statement and context.
+     * @param ryaStatement  The statement (triple) represented by this object.
+     * @param ryaContext    Context used for reading and writing the statement.
+     */
+    public RyaStatementWritable(RyaStatement ryaStatement, RyaTripleContext ryaContext) {
+        this(ryaContext);
+        this.ryaStatement = ryaStatement;
+    }
+
+    /**
+     * Gets the contained RyaStatement.
+     * @return The statement represented by this RyaStatementWritable.
+     */
+    public RyaStatement getRyaStatement() {
+        return ryaStatement;
+    }
+    /**
+     * Sets the contained RyaStatement.
+     * @param   ryaStatement    The statement to be represented by this
+     *                          RyaStatementWritable.
+     */
+    public void setRyaStatement(RyaStatement ryaStatement) {
+        this.ryaStatement = ryaStatement;
+    }
+
+    /**
+     * Comparison method for natural ordering. Compares based on the logical
+     * triple (the s/p/o/context information in the underlying RyaStatement)
+     * and then by the metadata contained in the RyaStatement if the triples are
+     * the same.
+     * @return  Zero if both RyaStatementWritables contain equivalent statements
+     *          or both have null statements; otherwise, an integer whose sign
+     *          corresponds to a consistent ordering.
+     */
+    @Override
+    public int compareTo(RyaStatementWritable other) {
+        CompareToBuilder builder = new CompareToBuilder();
+        RyaStatement rsThis = this.getRyaStatement();
+        RyaStatement rsOther = other.getRyaStatement(); // should throw NPE if other is null, as per Comparable contract
+        builder.append(rsThis == null, rsOther == null);
+        if (rsThis != null && rsOther != null) {
+            builder.append(rsThis.getSubject(), rsOther.getSubject());
+            builder.append(rsThis.getPredicate(), rsOther.getPredicate());
+            builder.append(rsThis.getObject(), rsOther.getObject());
+            builder.append(rsThis.getContext(), rsOther.getContext());
+            builder.append(rsThis.getQualifer(), rsOther.getQualifer());
+            builder.append(rsThis.getColumnVisibility(), rsOther.getColumnVisibility());
+            builder.append(rsThis.getValue(), rsOther.getValue());
+            builder.append(rsThis.getTimestamp(), rsOther.getTimestamp());
+        }
+        return builder.toComparison();
+    }
+
+    /**
+     * Returns a hash based on the hashCode method in RyaStatement.
+     * @return  A hash that should be consistent for equivalent RyaStatements.
+     */
+    @Override
+    public int hashCode() {
+        if (ryaStatement == null) {
+            return 0;
+        }
+        return ryaStatement.hashCode();
+    }
+
+    /**
+     * Tests for equality using the equals method in RyaStatement.
+     * @param   o   Object to compare with
+     * @return  true if both objects are RyaStatementWritables containing
+     *          equivalent RyaStatements.
+     */
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (o == null || !(o instanceof RyaStatementWritable)) {
+            return false;
+        }
+        RyaStatement rsThis = this.getRyaStatement();
+        RyaStatement rsOther = ((RyaStatementWritable) o).getRyaStatement();
+        if (rsThis == null) {
+            return rsOther == null;
+        }
+        else {
+            return rsThis.equals(rsOther);
+        }
+    }
+
+    /**
+     * Serializes this RyaStatementWritable.
+     * @param   dataOutput  An output stream for serialized statement data.
+     * @throws  IOException if the RyaStatement is null or otherwise can't be
+     *          serialized.
+     */
+    @Override
+    public void write(DataOutput dataOutput) throws IOException {
+        if (ryaStatement == null) {
+            throw new IOException("Rya Statement is null");
+        }
+        try {
+            Map<RdfCloudTripleStoreConstants.TABLE_LAYOUT, TripleRow> map = ryaContext.serializeTriple(ryaStatement);
+            TripleRow tripleRow = map.get(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO);
+            byte[] row = tripleRow.getRow();
+            byte[] columnFamily = tripleRow.getColumnFamily();
+            byte[] columnQualifier = tripleRow.getColumnQualifier();
+            write(dataOutput, row);
+            write(dataOutput, columnFamily);
+            write(dataOutput, columnQualifier);
+            write(dataOutput, ryaStatement.getColumnVisibility());
+            write(dataOutput, ryaStatement.getValue());
+            Long timestamp = ryaStatement.getTimestamp();
+            boolean b = timestamp != null;
+            dataOutput.writeBoolean(b);
+            if (b) {
+                dataOutput.writeLong(timestamp);
+            }
+        } catch (TripleRowResolverException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * Write part of a statement to an output stream.
+     * @param dataOutput Stream for writing serialized statements.
+     * @param row   Individual field to write, as a byte array.
+     * @throws IOException if writing to the stream fails.
+     */
+    protected void write(DataOutput dataOutput, byte[] row) throws IOException {
+        boolean b = row != null;
+        dataOutput.writeBoolean(b);
+        if (b) {
+            dataOutput.writeInt(row.length);
+            dataOutput.write(row);
+        }
+    }
+
+    /**
+     * Read part of a statement from an input stream.
+     * @param dataInput Stream for reading serialized statements.
+     * @return The next individual field, as a byte array.
+     * @throws IOException if reading from the stream fails.
+     */
+    protected byte[] read(DataInput dataInput) throws IOException {
+        if (dataInput.readBoolean()) {
+            int len = dataInput.readInt();
+            byte[] bytes = new byte[len];
+            dataInput.readFully(bytes);
+            return bytes;
+        }else {
+            return null;
+        }
+    }
+
+    /**
+     * Loads a RyaStatementWritable by reading data from an input stream.
+     * Creates a new RyaStatement and assigns it to this RyaStatementWritable.
+     * @param   dataInput   An stream containing serialized statement data.
+     */
+    @Override
+    public void readFields(DataInput dataInput) throws IOException {
+        byte[] row = read(dataInput);
+        byte[] columnFamily = read(dataInput);
+        byte[] columnQualifier = read(dataInput);
+        byte[] columnVisibility = read(dataInput);
+        byte[] value = read(dataInput);
+        boolean b = dataInput.readBoolean();
+        Long timestamp = null;
+        if (b) {
+            timestamp = dataInput.readLong();
+        }
+        try {
+            ryaStatement = ryaContext.deserializeTriple(RdfCloudTripleStoreConstants.TABLE_LAYOUT.SPO,
+                    new TripleRow(row, columnFamily, columnQualifier));
+            ryaStatement.setColumnVisibility(columnVisibility);
+            ryaStatement.setValue(value);
+            ryaStatement.setTimestamp(timestamp);
+        } catch (TripleRowResolverException e) {
+            throw new IOException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/examples/TextOutputExample.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/examples/TextOutputExample.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/examples/TextOutputExample.java
new file mode 100644
index 0000000..bc3af58
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/examples/TextOutputExample.java
@@ -0,0 +1,196 @@
+package mvm.rya.accumulo.mr.examples;
+
+import java.io.BufferedReader;
+
+/*
+ * 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 java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.Charset;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.util.Date;
+
+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.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Logger;
+import org.openrdf.model.Statement;
+import org.openrdf.rio.RDFFormat;
+import org.openrdf.rio.RDFHandlerException;
+import org.openrdf.rio.RDFWriter;
+import org.openrdf.rio.Rio;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
+import mvm.rya.accumulo.mr.MRUtils;
+import mvm.rya.accumulo.mr.RyaStatementWritable;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.api.resolver.RyaToRdfConversions;
+
+/**
+ * Example of using a MapReduce tool to get triples from a Rya instance and serialize them to a text file as RDF.
+ */
+public class TextOutputExample extends AbstractAccumuloMRTool {
+    private static Logger logger = Logger.getLogger(TextOutputExample.class);
+    private static RDFFormat rdfFormat = RDFFormat.NQUADS;
+    private static String tempDir;
+
+    // Connection information
+    private static final String USERNAME = "root";
+    private static final String PASSWORD = "";
+    private static final String INSTANCE_NAME = "instanceName";
+    private static final String PREFIX = "rya_example_";
+
+    public static void main(String[] args) throws Exception {
+        setUpRya();
+        TextOutputExample tool = new TextOutputExample();
+        ToolRunner.run(new Configuration(), tool, args);
+    }
+
+    static void setUpRya() throws AccumuloException, AccumuloSecurityException, RyaDAOException {
+        MockInstance mock = new MockInstance(INSTANCE_NAME);
+        Connector conn = mock.getConnector(USERNAME, new PasswordToken(PASSWORD));
+        AccumuloRyaDAO dao = new AccumuloRyaDAO();
+        dao.setConnector(conn);
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix(PREFIX);
+        dao.setConf(conf);
+        dao.init();
+        String ns = "http://example.com/";
+        dao.add(new RyaStatement(new RyaURI(ns+"s1"), new RyaURI(ns+"p1"), new RyaURI(ns+"o1")));
+        dao.add(new RyaStatement(new RyaURI(ns+"s1"), new RyaURI(ns+"p2"), new RyaURI(ns+"o2")));
+        dao.add(new RyaStatement(new RyaURI(ns+"s2"), new RyaURI(ns+"p1"), new RyaURI(ns+"o3"),
+                new RyaURI(ns+"g1")));
+        dao.add(new RyaStatement(new RyaURI(ns+"s3"), new RyaURI(ns+"p3"), new RyaURI(ns+"o3"),
+                new RyaURI(ns+"g2")));
+        dao.destroy();
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        logger.info("Configuring tool to connect to mock instance...");
+        MRUtils.setACUserName(conf, USERNAME);
+        MRUtils.setACPwd(conf, PASSWORD);
+        MRUtils.setACInstance(conf, INSTANCE_NAME);
+        MRUtils.setACMock(conf, true);
+        MRUtils.setTablePrefix(conf, PREFIX);
+
+        logger.info("Initializing tool and checking configuration...");
+        init();
+
+        logger.info("Creating Job, setting Mapper class, and setting no Reducer...");
+        Job job = Job.getInstance(conf);
+        job.setJarByClass(TextOutputExample.class);
+        job.setMapperClass(RyaToRdfMapper.class);
+        job.setNumReduceTasks(0);
+
+        logger.info("Configuring Job to take input from the mock Rya instance...");
+        setupRyaInput(job);
+
+        logger.info("Configuring Job to output Text to a new temporary directory...");
+        tempDir = Files.createTempDirectory("rya-mr-example").toString();
+        Path outputPath = new Path(tempDir, "rdf-output");
+        job.setOutputFormatClass(TextOutputFormat.class);
+        TextOutputFormat.setOutputPath(job, outputPath);
+        job.setOutputKeyClass(NullWritable.class);
+        job.setOutputValueClass(Text.class);
+
+        Date start = new Date();
+        logger.info("Starting Job at: start");
+        boolean success = job.waitForCompletion(true);
+
+        if (!success) {
+            System.out.println("Job Failed!!!");
+            return 1;
+        }
+
+        Date end = new Date();
+        logger.info("Job ended: " + end);
+        logger.info("The job took " + (end.getTime() - start.getTime()) / 1000 + " seconds.");
+        // Print output and then delete temp files:
+        java.nio.file.Path tempPath = FileSystems.getDefault().getPath(tempDir);
+        for (java.nio.file.Path subdir : Files.newDirectoryStream(tempPath)) {
+            logger.info("");
+            logger.info("Output files:");
+            for (java.nio.file.Path outputFile : Files.newDirectoryStream(subdir)) {
+                logger.info("\t" + outputFile);
+            }
+            for (java.nio.file.Path outputFile : Files.newDirectoryStream(subdir, "part*")) {
+                logger.info("");
+                logger.info("Contents of " + outputFile + ":");
+                BufferedReader reader = Files.newBufferedReader(outputFile, Charset.defaultCharset());
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    logger.info("\t" + line);
+                }
+                reader.close();
+            }
+            for (java.nio.file.Path outputFile : Files.newDirectoryStream(subdir)) {
+                Files.deleteIfExists(outputFile);
+            }
+            Files.deleteIfExists(subdir);
+        }
+        Files.deleteIfExists(tempPath);
+        logger.info("");
+        logger.info("Temporary directory " + tempDir + " deleted.");
+
+        return 0;
+    }
+
+    static class RyaToRdfMapper extends Mapper<Text, RyaStatementWritable, NullWritable, Text> {
+        Text textOut = new Text();
+        @Override
+        protected void map(Text key, RyaStatementWritable value, Context context) throws IOException, InterruptedException {
+            // receives a RyaStatementWritable; convert to a Statement
+            RyaStatement rstmt = value.getRyaStatement();
+            Statement st = RyaToRdfConversions.convertStatement(rstmt);
+            logger.info("Mapper receives: " + rstmt);
+            // then convert to an RDF string
+            StringWriter writer = new StringWriter();
+            try {
+                RDFWriter rdfWriter = Rio.createWriter(rdfFormat, writer);
+                rdfWriter.startRDF();
+                rdfWriter.handleStatement(st);
+                rdfWriter.endRDF();
+            } catch (RDFHandlerException e) {
+                throw new IOException("Error writing RDF data", e);
+            }
+            // Write the string to the output
+            String line = writer.toString().trim();
+            logger.info("Serialized to RDF: " + line);
+            textOut.set(line);
+            context.write(NullWritable.get(), textOut);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountTool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountTool.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountTool.java
new file mode 100644
index 0000000..ee4e00b
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/AccumuloRdfCountTool.java
@@ -0,0 +1,258 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 java.io.IOException;
+import java.util.Date;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRdfConstants;
+import mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
+import mvm.rya.accumulo.mr.MRUtils;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.resolver.RyaTripleContext;
+import mvm.rya.api.resolver.triple.TripleRow;
+import mvm.rya.api.resolver.triple.TripleRowResolverException;
+
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+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.ColumnVisibility;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.Lists;
+import com.google.common.io.ByteArrayDataInput;
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
+
+/**
+ * Count subject, predicate, object. Save in table
+ * Class RdfCloudTripleStoreCountTool
+ * Date: Apr 12, 2011
+ * Time: 10:39:40 AM
+ * @deprecated
+ */
+public class AccumuloRdfCountTool extends AbstractAccumuloMRTool implements Tool {
+
+    public static void main(String[] args) {
+        try {
+
+            ToolRunner.run(new Configuration(), new AccumuloRdfCountTool(), args);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    /**
+     * cloudbase props
+     */
+
+    @Override
+    public int run(String[] strings) throws Exception {
+        conf.set(MRUtils.JOB_NAME_PROP, "Gather Evaluation Statistics");
+
+        //initialize
+        init();
+
+        Job job = new Job(conf);
+        job.setJarByClass(AccumuloRdfCountTool.class);
+        setupAccumuloInput(job);
+
+        AccumuloInputFormat.setRanges(job, Lists.newArrayList(new Range(new Text(new byte[]{}), new Text(new byte[]{Byte.MAX_VALUE}))));
+        // set input output of the particular job
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(Mutation.class);
+
+        // set mapper and reducer classes
+        job.setMapperClass(CountPiecesMapper.class);
+        job.setCombinerClass(CountPiecesCombiner.class);
+        job.setReducerClass(CountPiecesReducer.class);
+
+        String outputTable = MRUtils.getTablePrefix(conf) + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX;
+        setupAccumuloOutput(job, outputTable);
+
+        // Submit the job
+        Date startTime = new Date();
+        System.out.println("Job started: " + startTime);
+        int exitCode = job.waitForCompletion(true) ? 0 : 1;
+
+        if (exitCode == 0) {
+            Date end_time = new Date();
+            System.out.println("Job ended: " + end_time);
+            System.out.println("The job took "
+                    + (end_time.getTime() - startTime.getTime()) / 1000
+                    + " seconds.");
+            return 0;
+        } else {
+            System.out.println("Job Failed!!!");
+        }
+
+        return -1;
+    }
+
+    public static class CountPiecesMapper extends Mapper<Key, Value, Text, LongWritable> {
+
+        public static final byte[] EMPTY_BYTES = new byte[0];
+        private RdfCloudTripleStoreConstants.TABLE_LAYOUT tableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP;
+
+        ValueFactoryImpl vf = new ValueFactoryImpl();
+
+        private Text keyOut = new Text();
+        private LongWritable valOut = new LongWritable(1);
+        private RyaTripleContext ryaContext;
+
+        @Override
+        protected void setup(Context context) throws IOException, InterruptedException {
+            super.setup(context);
+            Configuration conf = context.getConfiguration();
+            tableLayout = RdfCloudTripleStoreConstants.TABLE_LAYOUT.valueOf(
+                    conf.get(MRUtils.TABLE_LAYOUT_PROP, RdfCloudTripleStoreConstants.TABLE_LAYOUT.OSP.toString()));
+            ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration(conf));
+        }
+
+        @Override
+        protected void map(Key key, Value value, Context context) throws IOException, InterruptedException {
+            try {
+                RyaStatement statement = ryaContext.deserializeTriple(tableLayout, new TripleRow(key.getRow().getBytes(), key.getColumnFamily().getBytes(), key.getColumnQualifier().getBytes()));
+                //count each piece subject, pred, object
+
+                String subj = statement.getSubject().getData();
+                String pred = statement.getPredicate().getData();
+//                byte[] objBytes = tripleFormat.getValueFormat().serialize(statement.getObject());
+                RyaURI scontext = statement.getContext();
+                boolean includesContext = scontext != null;
+                String scontext_str = (includesContext) ? scontext.getData() : null;
+
+                ByteArrayDataOutput output = ByteStreams.newDataOutput();
+                output.writeUTF(subj);
+                output.writeUTF(RdfCloudTripleStoreConstants.SUBJECT_CF);
+                output.writeBoolean(includesContext);
+                if (includesContext)
+                    output.writeUTF(scontext_str);
+                keyOut.set(output.toByteArray());
+                context.write(keyOut, valOut);
+
+                output = ByteStreams.newDataOutput();
+                output.writeUTF(pred);
+                output.writeUTF(RdfCloudTripleStoreConstants.PRED_CF);
+                output.writeBoolean(includesContext);
+                if (includesContext)
+                    output.writeUTF(scontext_str);
+                keyOut.set(output.toByteArray());
+                context.write(keyOut, valOut);
+            } catch (TripleRowResolverException e) {
+                throw new IOException(e);
+            }
+        }
+    }
+
+    public static class CountPiecesCombiner extends Reducer<Text, LongWritable, Text, LongWritable> {
+
+        private LongWritable valOut = new LongWritable();
+
+        // TODO: can still add up to be large I guess
+        // any count lower than this does not need to be saved
+        public static final int TOO_LOW = 2;
+
+        @Override
+        protected void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+            long count = 0;
+            for (LongWritable lw : values) {
+                count += lw.get();
+            }
+
+            if (count <= TOO_LOW)
+                return;
+
+            valOut.set(count);
+            context.write(key, valOut);
+        }
+
+    }
+
+    public static class CountPiecesReducer extends Reducer<Text, LongWritable, Text, Mutation> {
+
+        Text row = new Text();
+        Text cat_txt = new Text();
+        Value v_out = new Value();
+        ValueFactory vf = new ValueFactoryImpl();
+
+        // any count lower than this does not need to be saved
+        public static final int TOO_LOW = 10;
+        private String tablePrefix;
+        protected Text table;
+        private ColumnVisibility cv = AccumuloRdfConstants.EMPTY_CV;
+
+        @Override
+        protected void setup(Context context) throws IOException, InterruptedException {
+            super.setup(context);
+            tablePrefix = context.getConfiguration().get(MRUtils.TABLE_PREFIX_PROPERTY, RdfCloudTripleStoreConstants.TBL_PRFX_DEF);
+            table = new Text(tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX);
+            final String cv_s = context.getConfiguration().get(MRUtils.AC_CV_PROP);
+            if (cv_s != null)
+                cv = new ColumnVisibility(cv_s);
+        }
+
+        @Override
+        protected void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+            long count = 0;
+            for (LongWritable lw : values) {
+                count += lw.get();
+            }
+
+            if (count <= TOO_LOW)
+                return;
+
+            ByteArrayDataInput badi = ByteStreams.newDataInput(key.getBytes());
+            String v = badi.readUTF();
+            cat_txt.set(badi.readUTF());
+
+            Text columnQualifier = RdfCloudTripleStoreConstants.EMPTY_TEXT;
+            boolean includesContext = badi.readBoolean();
+            if (includesContext) {
+                columnQualifier = new Text(badi.readUTF());
+            }
+
+            row.set(v);
+            Mutation m = new Mutation(row);
+            v_out.set((count + "").getBytes());
+            m.put(cat_txt, columnQualifier, cv, v_out);
+            context.write(table, m);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/RdfFileInputTool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/RdfFileInputTool.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/RdfFileInputTool.java
new file mode 100644
index 0000000..7857e45
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/RdfFileInputTool.java
@@ -0,0 +1,91 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 java.util.Date;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.openrdf.rio.RDFFormat;
+
+import mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
+import mvm.rya.accumulo.mr.MRUtils;
+
+/**
+ * Reads RDF data from one or more file(s) and inserts statements into Rya.
+ * <p>
+ * Uses {@link mvm.rya.accumulo.mr.RdfFileInputFormat} to read data.
+ * <p>
+ * Takes one argument: the file or directory to read (from HDFS).
+ * <p>
+ * Expects configuration:
+ * <p>
+ * - RDF format, named by parameter "rdf.format"; see {@link RDFFormat}.
+ *   Defaults to rdf/xml. If using multiple files, all must be the same format.
+ * <p>
+ * - Accumulo and Rya configuration parameters as named in {@link MRUtils}
+ *   (username, password, instance name, zookeepers, and Rya prefix)
+ * <p>
+ * - Indexing configuration parameters as named in
+ *   {@link mvm.rya.indexing.accumulo.ConfigUtils} (enable or disable freetext,
+ *   geo, temporal, and entity indexing, and specify predicates for each
+ *   indexer). If not given, no secondary indexing is done.
+ */
+public class RdfFileInputTool extends AbstractAccumuloMRTool implements Tool {
+    public static void main(String[] args) {
+        try {
+            ToolRunner.run(new Configuration(), new RdfFileInputTool(), args);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        init();
+        Job job = Job.getInstance(conf, "Rdf File Input");
+        job.setJarByClass(RdfFileInputTool.class);
+
+        String inputPath = conf.get(MRUtils.INPUT_PATH, args[0]);
+        setupFileInput(job, inputPath, RDFFormat.RDFXML);
+        setupRyaOutput(job);
+        job.setNumReduceTasks(0);
+
+        Date startTime = new Date();
+        System.out.println("Job started: " + startTime);
+        int exitCode = job.waitForCompletion(true) ? 0 : 1;
+
+        if (exitCode == 0) {
+            Date end_time = new Date();
+            System.out.println("Job ended: " + end_time);
+            System.out.println("The job took "
+                    + (end_time.getTime() - startTime.getTime()) / 1000
+                    + " seconds.");
+            long n = job.getCounters()
+                    .findCounter("org.apache.hadoop.mapred.Task$Counter", "MAP_OUTPUT_RECORDS").getValue();
+            System.out.println(n + " statement(s) inserted to Rya.");
+        } else {
+            System.out.println("Job Failed!!!");
+        }
+        return exitCode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/Upgrade322Tool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/Upgrade322Tool.java b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/Upgrade322Tool.java
new file mode 100644
index 0000000..d713d85
--- /dev/null
+++ b/mapreduce/src/main/java/mvm/rya/accumulo/mr/tools/Upgrade322Tool.java
@@ -0,0 +1,241 @@
+package mvm.rya.accumulo.mr.tools;
+
+/*
+ * 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 mvm.rya.accumulo.mr.AbstractAccumuloMRTool;
+import mvm.rya.accumulo.mr.MRUtils;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+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.iterators.user.RegExFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.calrissian.mango.types.LexiTypeEncoders;
+import org.calrissian.mango.types.TypeEncoder;
+
+import java.io.IOException;
+import java.util.Date;
+
+import static mvm.rya.api.RdfCloudTripleStoreConstants.*;
+
+/**
+ */
+public class Upgrade322Tool extends AbstractAccumuloMRTool implements Tool {
+    @Override
+    public int run(String[] strings) throws Exception {
+        conf.set(MRUtils.JOB_NAME_PROP, "Upgrade to Rya 3.2.2");
+        //faster
+        init();
+
+        Job job = new Job(conf);
+        job.setJarByClass(Upgrade322Tool.class);
+
+        setupAccumuloInput(job);
+        AccumuloInputFormat.setInputTableName(job, MRUtils.getTablePrefix(conf) + TBL_OSP_SUFFIX);
+
+        //we do not need to change any row that is a string, custom, or uri type
+        IteratorSetting regex = new IteratorSetting(30, "regex",
+                                                    RegExFilter.class);
+        RegExFilter.setRegexs(regex, "\\w*" + TYPE_DELIM + "[\u0003|\u0008|\u0002]", null, null, null, false);
+        RegExFilter.setNegate(regex, true);
+
+        // set input output of the particular job
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Mutation.class);
+
+        setupAccumuloOutput(job, MRUtils.getTablePrefix(conf) +
+                               TBL_SPO_SUFFIX);
+
+        // set mapper and reducer classes
+        job.setMapperClass(Upgrade322Mapper.class);
+        job.setReducerClass(Reducer.class);
+
+        // Submit the job
+        return job.waitForCompletion(true) ? 0 : 1;
+    }
+
+    public static void main(String[] args) {
+        try {
+            ToolRunner.run(new Configuration(), new Upgrade322Tool(), args);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    /**
+     * Reading from the OSP table
+     */
+    public static class Upgrade322Mapper extends Mapper<Key, Value, Text, Mutation> {
+
+        private String tablePrefix;
+        private Text spoTable;
+        private Text poTable;
+        private Text ospTable;
+
+        private final UpgradeObjectSerialization upgradeObjectSerialization;
+
+        public Upgrade322Mapper() {
+            this(new UpgradeObjectSerialization());
+        }
+
+        public Upgrade322Mapper(
+          UpgradeObjectSerialization upgradeObjectSerialization) {
+            this.upgradeObjectSerialization = upgradeObjectSerialization;
+        }
+
+        @Override
+        protected void setup(
+          Context context) throws IOException, InterruptedException {
+            super.setup(context);
+
+            tablePrefix = context.getConfiguration().get(
+              MRUtils.TABLE_PREFIX_PROPERTY, TBL_PRFX_DEF);
+            spoTable = new Text(tablePrefix + TBL_SPO_SUFFIX);
+            poTable = new Text(tablePrefix + TBL_PO_SUFFIX);
+            ospTable = new Text(tablePrefix + TBL_OSP_SUFFIX);
+        }
+
+        @Override
+        protected void map(
+          Key key, Value value, Context context)
+          throws IOException, InterruptedException {
+
+            //read the key, expect OSP
+            final String row = key.getRow().toString();
+            final int firstDelim = row.indexOf(DELIM);
+            final int secondDelim = row.indexOf(DELIM, firstDelim + 1);
+            final int typeDelim = row.lastIndexOf(TYPE_DELIM);
+            final String oldSerialization = row.substring(0, firstDelim);
+            char typeMarker = row.charAt(row.length() - 1);
+
+            final String subject = row.substring(firstDelim + 1, secondDelim);
+            final String predicate = row.substring(secondDelim + 1, typeDelim);
+            final String typeSuffix = TYPE_DELIM + typeMarker;
+
+            String newSerialization = upgradeObjectSerialization.upgrade(oldSerialization, typeMarker);
+            if(newSerialization == null) {
+                return;
+            }
+
+            //write out delete Mutations
+            Mutation deleteOldSerialization_osp = new Mutation(key.getRow());
+            deleteOldSerialization_osp.putDelete(key.getColumnFamily(), key.getColumnQualifier(),
+                               key.getColumnVisibilityParsed());
+            Mutation deleteOldSerialization_po = new Mutation(predicate + DELIM + oldSerialization + DELIM + subject + typeSuffix);
+            deleteOldSerialization_po.putDelete(key.getColumnFamily(),
+                                                key.getColumnQualifier(),
+                                                key.getColumnVisibilityParsed());
+            Mutation deleteOldSerialization_spo = new Mutation(subject + DELIM + predicate + DELIM + oldSerialization + typeSuffix);
+            deleteOldSerialization_spo.putDelete(key.getColumnFamily(), key.getColumnQualifier(),
+                                                key.getColumnVisibilityParsed());
+
+            //write out new serialization
+            Mutation putNewSerialization_osp = new Mutation(newSerialization + DELIM + subject + DELIM + predicate + typeSuffix);
+            putNewSerialization_osp.put(key.getColumnFamily(),
+                                        key.getColumnQualifier(),
+                                        key.getColumnVisibilityParsed(),
+                                        key.getTimestamp(), value);
+            Mutation putNewSerialization_po = new Mutation(predicate + DELIM + newSerialization + DELIM + subject + typeSuffix);
+            putNewSerialization_po.put(key.getColumnFamily(),
+                                       key.getColumnQualifier(),
+                                       key.getColumnVisibilityParsed(),
+                                       key.getTimestamp(), value);
+            Mutation putNewSerialization_spo = new Mutation(subject + DELIM + predicate + DELIM + newSerialization + typeSuffix);
+            putNewSerialization_spo.put(key.getColumnFamily(),
+                                        key.getColumnQualifier(),
+                                        key.getColumnVisibilityParsed(),
+                                        key.getTimestamp(), value);
+
+            //write out deletes to all tables
+            context.write(ospTable, deleteOldSerialization_osp);
+            context.write(poTable, deleteOldSerialization_po);
+            context.write(spoTable, deleteOldSerialization_spo);
+
+            //write out inserts to all tables
+            context.write(ospTable, putNewSerialization_osp);
+            context.write(poTable, putNewSerialization_po);
+            context.write(spoTable, putNewSerialization_spo);
+        }
+    }
+
+    public static class UpgradeObjectSerialization {
+
+        public static final TypeEncoder<Boolean, String>
+          BOOLEAN_STRING_TYPE_ENCODER = LexiTypeEncoders.booleanEncoder();
+        public static final TypeEncoder<Byte, String> BYTE_STRING_TYPE_ENCODER
+          = LexiTypeEncoders.byteEncoder();
+        public static final TypeEncoder<Date, String> DATE_STRING_TYPE_ENCODER
+          = LexiTypeEncoders.dateEncoder();
+        public static final TypeEncoder<Integer, String>
+          INTEGER_STRING_TYPE_ENCODER = LexiTypeEncoders.integerEncoder();
+        public static final TypeEncoder<Long, String> LONG_STRING_TYPE_ENCODER
+          = LexiTypeEncoders.longEncoder();
+        public static final TypeEncoder<Double, String>
+          DOUBLE_STRING_TYPE_ENCODER = LexiTypeEncoders.doubleEncoder();
+
+        public String upgrade(String object, int typeMarker) {
+            switch(typeMarker) {
+                case 10: //boolean
+                    final boolean bool = Boolean.parseBoolean(object);
+                    return BOOLEAN_STRING_TYPE_ENCODER.encode(bool);
+                case 9: //byte
+                    final byte b = Byte.parseByte(object);
+                    return BYTE_STRING_TYPE_ENCODER.encode(b);
+                case 4: //long
+                    final Long lng = Long.parseLong(object);
+                    return LONG_STRING_TYPE_ENCODER.encode(lng);
+                case 5: //int
+                    final Integer i = Integer.parseInt(object);
+                    return INTEGER_STRING_TYPE_ENCODER.encode(i);
+                case 6: //double
+                    String exp = object.substring(2, 5);
+                    char valueSign = object.charAt(0);
+                    char expSign = object.charAt(1);
+                    Integer expInt = Integer.parseInt(exp);
+                    if (expSign == '-') {
+                        expInt = 999 - expInt;
+                    }
+                    final String expDoubleStr =
+                      String.format("%s%sE%s%d", valueSign,
+                                    object.substring(6),
+                                    expSign, expInt);
+                    return DOUBLE_STRING_TYPE_ENCODER
+                      .encode(Double.parseDouble(expDoubleStr));
+                case 7: //datetime
+                    //check to see if it is an early release that includes the exact term xsd:dateTime
+                    final Long l = Long.MAX_VALUE - Long.parseLong(object);
+                    Date date = new Date(l);
+                    return DATE_STRING_TYPE_ENCODER.encode(date);
+                default:
+                    return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/RdfFileInputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/RdfFileInputFormatTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RdfFileInputFormatTest.java
new file mode 100644
index 0000000..cda66bd
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RdfFileInputFormatTest.java
@@ -0,0 +1,180 @@
+package mvm.rya.accumulo.mr;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.ContextStatementImpl;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.rio.RDFFormat;
+
+import mvm.rya.api.resolver.RyaToRdfConversions;
+
+/*
+ * 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.
+ */
+
+public class RdfFileInputFormatTest {
+    static String NT_INPUT = "src/test/resources/test.ntriples";
+    static String TRIG_INPUT = "src/test/resources/namedgraphs.trig";
+
+    Configuration conf;
+    Job job;
+    FileSystem fs;
+    RdfFileInputFormat.RdfFileRecordReader reader;
+
+    @Rule
+    public ExpectedException expected = ExpectedException.none();
+
+    @Before
+    public void before() throws IOException {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        fs = FileSystem.get(conf);
+        job = Job.getInstance(conf);
+    }
+
+    void init(String filename) throws IOException, InterruptedException {
+        conf = job.getConfiguration();
+        File inputFile = new File(filename);
+        Path inputPath = new Path(inputFile.getAbsoluteFile().toURI());
+        InputSplit split = new FileSplit(inputPath, 0, inputFile.length(), null);
+        TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
+        reader = (RdfFileInputFormat.RdfFileRecordReader) new RdfFileInputFormat().createRecordReader(split, context);
+        reader.initialize(split, context);
+    }
+
+    @Test
+    public void testStatementInput() throws Exception {
+        RdfFileInputFormat.setRDFFormat(job, RDFFormat.NTRIPLES);
+        init(NT_INPUT);
+        String prefix = "urn:lubm:rdfts#";
+        URI[] gs = {
+                new URIImpl(prefix + "GraduateStudent01"),
+                new URIImpl(prefix + "GraduateStudent02"),
+                new URIImpl(prefix + "GraduateStudent03"),
+                new URIImpl(prefix + "GraduateStudent04")
+        };
+        URI hasFriend = new URIImpl(prefix + "hasFriend");
+        Statement[] statements = {
+                new StatementImpl(gs[0], hasFriend, gs[1]),
+                new StatementImpl(gs[1], hasFriend, gs[2]),
+                new StatementImpl(gs[2], hasFriend, gs[3])
+        };
+        int count = 0;
+        while (reader.nextKeyValue()) {
+            Assert.assertEquals(statements[count],
+                    RyaToRdfConversions.convertStatement(reader.getCurrentValue().getRyaStatement()));
+            count++;
+            Assert.assertEquals(count, reader.getCurrentKey().get());
+        }
+        Assert.assertEquals(3, count);
+    }
+
+    @Test
+    public void testTrigInput() throws Exception {
+        RdfFileInputFormat.setRDFFormat(job, RDFFormat.TRIG);
+        init(TRIG_INPUT);
+        Assert.assertTrue(reader.nextKeyValue());
+        Assert.assertEquals(1, reader.getCurrentKey().get());
+        Statement expected = new ContextStatementImpl(
+            new URIImpl("http://www.example.org/exampleDocument#Monica"),
+            new URIImpl("http://www.example.org/vocabulary#name"),
+            new LiteralImpl("Monica Murphy"),
+            new URIImpl("http://www.example.org/exampleDocument#G1"));
+        Statement actual = RyaToRdfConversions.convertStatement(
+            reader.getCurrentValue().getRyaStatement());
+        Assert.assertEquals(expected, actual);
+    }
+
+    @Test
+    public void testBlockStatementQueue() throws Exception {
+        RdfFileInputFormat.setRDFFormat(job, RDFFormat.NTRIPLES);
+        RdfFileInputFormat.setStatementBufferSize(job, 2);
+        init(NT_INPUT);
+        // 3 statements in total, plus done signal: should fill up three times
+        int interval = 100; // ms to sleep per iteration while waiting for statement cache to fill
+        int maxSeconds = 120; // timeout that should never be reached
+        for (int i = 0; i < 3; i++) {
+            long t = 0;
+            while (reader.statementCache.remainingCapacity() > 0) {
+                if (t >= (maxSeconds*1000)) {
+                    Assert.fail("Statement buffer still hasn't filled up after " + maxSeconds + " seconds.");
+                }
+                Assert.assertTrue(reader.statementCache.size() <= 2);
+                Thread.sleep(interval);
+                t += interval;
+            }
+            Assert.assertEquals(2, reader.statementCache.size());
+            Assert.assertEquals(0, reader.statementCache.remainingCapacity());
+            Assert.assertTrue(reader.nextKeyValue());
+        }
+        // Then the only thing in the queue should be the done signal
+        Assert.assertSame(RdfFileInputFormat.DONE, reader.statementCache.peek());
+        Assert.assertEquals(1, reader.statementCache.size());
+        Assert.assertFalse(reader.nextKeyValue());
+        Assert.assertTrue(reader.statementCache.isEmpty());
+    }
+
+    @Test
+    public void testFailGracefully() throws Exception {
+        // Pass the wrong RDF format and make sure all threads terminate
+        int interval = 100; // ms to sleep per iteration while waiting for statement cache to fill
+        int maxSeconds = 60; // timeout that should never be reached
+        RdfFileInputFormat.setRDFFormat(job, RDFFormat.RDFXML);
+        RdfFileInputFormat.setTimeout(job, maxSeconds*2);
+        init(NT_INPUT);
+        long t = 0;
+        while (reader.statementCache.isEmpty()) {
+            if (t >= (maxSeconds*1000)) {
+                Assert.fail("Statement buffer still hasn't been sent error signal after " + maxSeconds + " seconds.");
+            }
+            Thread.sleep(interval);
+            t += interval;
+        }
+        Assert.assertSame(RdfFileInputFormat.ERROR, reader.statementCache.peek());
+        expected.expect(IOException.class);
+        try {
+            Assert.assertFalse(reader.nextKeyValue());
+        }
+        catch (Exception e) {
+            Assert.assertNull(reader.getCurrentKey());
+            Assert.assertNull(reader.getCurrentValue());
+            Assert.assertFalse(reader.readerThread.isAlive());
+            Assert.assertFalse(reader.parserThread.isAlive());
+            throw e;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
new file mode 100644
index 0000000..2755732
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaInputFormatTest.java
@@ -0,0 +1,156 @@
+package mvm.rya.accumulo.mr;
+/*
+ * 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 mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.accumulo.RyaTableMutationsFactory;
+import mvm.rya.accumulo.mr.RyaInputFormat.RyaStatementRecordReader;
+import mvm.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.resolver.RyaTripleContext;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+public class RyaInputFormatTest {
+
+    static String username = "root", table = "rya_spo";
+    static PasswordToken password = new PasswordToken("");
+
+    static Instance instance;
+    static AccumuloRyaDAO apiImpl;
+
+    @BeforeClass
+    public static void init() throws Exception {
+        instance = new MockInstance(RyaInputFormatTest.class.getName() + ".mock_instance");
+        Connector connector = instance.getConnector(username, password);
+        connector.tableOperations().create(table);
+
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix("rya_");
+        conf.setDisplayQueryPlan(false);
+
+        apiImpl = new AccumuloRyaDAO();
+        apiImpl.setConf(conf);
+        apiImpl.setConnector(connector);
+    }
+
+    @Before
+    public void before() throws Exception {
+        apiImpl.init();
+    }
+
+    @After
+    public void after() throws Exception {
+        apiImpl.dropAndDestroy();
+    }
+
+    @Test
+    public void testInputFormat() throws Exception {
+
+
+        RyaStatement input = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setColumnVisibility(new byte[0])
+            .setValue(new byte[0])
+            .build();
+
+        apiImpl.add(input);
+
+        Job jobConf = Job.getInstance();
+
+        RyaInputFormat.setMockInstance(jobConf, instance.getInstanceName());
+        RyaInputFormat.setConnectorInfo(jobConf, username, password);
+        RyaInputFormat.setTableLayout(jobConf, TABLE_LAYOUT.SPO);
+
+        AccumuloInputFormat.setInputTableName(jobConf, table);
+        AccumuloInputFormat.setInputTableName(jobConf, table);
+        AccumuloInputFormat.setScanIsolation(jobConf, false);
+        AccumuloInputFormat.setLocalIterators(jobConf, false);
+        AccumuloInputFormat.setOfflineTableScan(jobConf, false);
+
+        RyaInputFormat inputFormat = new RyaInputFormat();
+
+        JobContext context = new JobContextImpl(jobConf.getConfiguration(), jobConf.getJobID());
+
+        List<InputSplit> splits = inputFormat.getSplits(context);
+
+        Assert.assertEquals(1, splits.size());
+
+        TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(context.getConfiguration(), new TaskAttemptID(new TaskID(), 1));
+
+        RecordReader<Text, RyaStatementWritable> reader = inputFormat.createRecordReader(splits.get(0), taskAttemptContext);
+
+        RyaStatementRecordReader ryaStatementRecordReader = (RyaStatementRecordReader)reader;
+        ryaStatementRecordReader.initialize(splits.get(0), taskAttemptContext);
+
+        List<RyaStatement> results = new ArrayList<RyaStatement>();
+        while(ryaStatementRecordReader.nextKeyValue()) {
+            RyaStatementWritable writable = ryaStatementRecordReader.getCurrentValue();
+            RyaStatement value = writable.getRyaStatement();
+            Text text = ryaStatementRecordReader.getCurrentKey();
+            RyaStatement stmt = RyaStatement.builder()
+                .setSubject(value.getSubject())
+                .setPredicate(value.getPredicate())
+                .setObject(value.getObject())
+                .setContext(value.getContext())
+                .setQualifier(value.getQualifer())
+                .setColumnVisibility(value.getColumnVisibility())
+                .setValue(value.getValue())
+                .build();
+            results.add(stmt);
+
+            System.out.println(text);
+            System.out.println(value);
+        }
+
+        Assert.assertTrue(results.size() == 2);
+        Assert.assertTrue(results.contains(input));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaOutputFormatTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaOutputFormatTest.java
new file mode 100644
index 0000000..a48afa3
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaOutputFormatTest.java
@@ -0,0 +1,324 @@
+package mvm.rya.accumulo.mr;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+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.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.Statement;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.XMLSchema;
+
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.Point;
+import com.vividsolutions.jts.geom.PrecisionModel;
+
+import info.aduna.iteration.CloseableIteration;
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaType;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.resolver.RdfToRyaConversions;
+import mvm.rya.api.resolver.RyaToRdfConversions;
+import mvm.rya.api.resolver.RyaTripleContext;
+import mvm.rya.indexing.StatementConstraints;
+import mvm.rya.indexing.TemporalInstant;
+import mvm.rya.indexing.TemporalInstantRfc3339;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.accumulo.entity.EntityCentricIndex;
+import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
+import mvm.rya.indexing.accumulo.freetext.SimpleTokenizer;
+import mvm.rya.indexing.accumulo.freetext.Tokenizer;
+import mvm.rya.indexing.accumulo.geo.GeoConstants;
+import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
+import mvm.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
+
+/*
+ * 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.
+ */
+
+public class RyaOutputFormatTest {
+    private static final String CV = "test_auth";
+    private static final String GRAPH = "http://example.org/graph";
+    private static final String USERNAME = "root";
+    private static final String PASSWORD = "";
+    private static final String INSTANCE_NAME = RyaOutputFormatTest.class.getSimpleName() + ".rya_output";
+    private static final String PREFIX = "ryaoutputformattest_";
+
+    MockInstance instance;
+    Connector connector;
+    AccumuloRyaDAO dao;
+    AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+    Job job;
+    RyaTripleContext ryaContext;
+
+    @Before
+    public void init() throws Exception {
+        MRUtils.setACMock(conf, true);
+        MRUtils.setACInstance(conf, INSTANCE_NAME);
+        MRUtils.setACUserName(conf, USERNAME);
+        MRUtils.setACPwd(conf, PASSWORD);
+        MRUtils.setTablePrefix(conf, PREFIX);
+        conf.setTablePrefix(PREFIX);
+        conf.setAuths(CV);
+        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, INSTANCE_NAME);
+        conf.set(ConfigUtils.CLOUDBASE_USER, USERNAME);
+        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, PASSWORD);
+        conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, true);
+        conf.setClass(ConfigUtils.TOKENIZER_CLASS, SimpleTokenizer.class, Tokenizer.class);
+        ryaContext = RyaTripleContext.getInstance(conf);
+        instance = new MockInstance(INSTANCE_NAME);
+        connector = instance.getConnector(USERNAME, new PasswordToken(PASSWORD));
+        job = Job.getInstance(conf);
+        RyaOutputFormat.setMockInstance(job, instance.getInstanceName());
+        AccumuloOutputFormat.setConnectorInfo(job, USERNAME, new PasswordToken(PASSWORD));
+        AccumuloOutputFormat.setCreateTables(job, true);
+        AccumuloOutputFormat.setDefaultTableName(job, PREFIX + "default");
+        RyaOutputFormat.setTablePrefix(job, PREFIX);
+    }
+
+    private void write(RyaStatement... input) throws IOException, InterruptedException {
+        RecordWriter<Writable, RyaStatementWritable> writer =
+                new RyaOutputFormat.RyaRecordWriter(job.getConfiguration());
+        for (RyaStatement rstmt : input) {
+            RyaStatementWritable rsw = new RyaStatementWritable(rstmt, ryaContext);
+            writer.write(new Text("unused"), rsw);
+        }
+        writer.close(null);
+    }
+
+    @Test
+    public void testOutputFormat() throws Exception {
+        RyaStatement input = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setColumnVisibility(CV.getBytes())
+            .setValue(new byte[0])
+            .setContext(new RyaURI(GRAPH))
+            .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, true);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        write(input);
+        TestUtils.verify(connector, conf, input);
+    }
+
+    @Test
+    public void testDefaultCV() throws Exception {
+        RyaStatement input = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setValue(new byte[0])
+            .setContext(new RyaURI(GRAPH))
+            .build();
+        RyaStatement expected = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setValue(new byte[0])
+            .setContext(new RyaURI(GRAPH))
+            .setColumnVisibility(CV.getBytes())
+            .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, true);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        RyaOutputFormat.setDefaultVisibility(job, CV);
+        write(input);
+        TestUtils.verify(connector, conf, expected);
+    }
+
+    @Test
+    public void testDefaultGraph() throws Exception {
+        RyaStatement input = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setValue(new byte[0])
+            .setColumnVisibility(CV.getBytes())
+            .build();
+        RyaStatement expected = RyaStatement.builder()
+            .setSubject(new RyaURI("http://www.google.com"))
+            .setPredicate(new RyaURI("http://some_other_uri"))
+            .setObject(new RyaURI("http://www.yahoo.com"))
+            .setValue(new byte[0])
+            .setColumnVisibility(CV.getBytes())
+            .setContext(new RyaURI(GRAPH))
+            .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, true);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        RyaOutputFormat.setDefaultContext(job, GRAPH);
+        write(input);
+        TestUtils.verify(connector, conf, expected);
+    }
+
+    @Test
+    public void testFreeTextIndexing() throws Exception {
+        AccumuloFreeTextIndexer ft = new AccumuloFreeTextIndexer();
+        ft.setConf(conf);
+        RyaStatement input = RyaStatement.builder()
+                .setSubject(new RyaURI(GRAPH + ":s"))
+                .setPredicate(new RyaURI(GRAPH + ":p"))
+                .setObject(new RyaType(XMLSchema.STRING, "one two three four five"))
+                .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, false);
+        RyaOutputFormat.setFreeTextEnabled(job, true);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        write(input);
+        Set<Statement> empty = new HashSet<>();
+        Set<Statement> expected = new HashSet<>();
+        expected.add(RyaToRdfConversions.convertStatement(input));
+        Assert.assertEquals(expected, getSet(ft.queryText("one", new StatementConstraints())));
+        Assert.assertEquals(empty, getSet(ft.queryText("!two", new StatementConstraints())));
+        Assert.assertEquals(expected, getSet(ft.queryText("*r", new StatementConstraints())));
+        Assert.assertEquals(empty, getSet(ft.queryText("r*", new StatementConstraints())));
+        Assert.assertEquals(expected, getSet(ft.queryText("!r*", new StatementConstraints())));
+        Assert.assertEquals(expected, getSet(ft.queryText("t* & !s*", new StatementConstraints())));
+        ft.close();
+    }
+
+    @Test
+    public void testTemporalIndexing() throws Exception {
+        TemporalInstant[] instants = {
+                new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 01),
+                new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 02),
+                new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 03),
+                new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 03)
+        };
+        Statement[] statements = new Statement[instants.length];
+        RyaOutputFormat.setCoreTablesEnabled(job, false);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, true);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        ValueFactory vf = new ValueFactoryImpl();
+        for (int i = 0; i < instants.length; i++) {
+            RyaType time = RdfToRyaConversions.convertLiteral(vf.createLiteral(instants[i].toString()));
+            RyaStatement input = RyaStatement.builder()
+                    .setSubject(new RyaURI(GRAPH + ":s"))
+                    .setPredicate(new RyaURI(GRAPH + ":p"))
+                    .setObject(time)
+                    .build();
+            write(input);
+            statements[i] = RyaToRdfConversions.convertStatement(input);
+        }
+        AccumuloTemporalIndexer temporal = new AccumuloTemporalIndexer();
+        temporal.setConf(conf);
+        Set<Statement> empty = new HashSet<>();
+        Set<Statement> head = new HashSet<>();
+        Set<Statement> tail = new HashSet<>();
+        head.add(statements[0]);
+        tail.add(statements[2]);
+        tail.add(statements[3]);
+        Assert.assertEquals(empty, getSet(temporal.queryInstantBeforeInstant(instants[0], new StatementConstraints())));
+        Assert.assertEquals(empty, getSet(temporal.queryInstantAfterInstant(instants[3], new StatementConstraints())));
+        Assert.assertEquals(head, getSet(temporal.queryInstantBeforeInstant(instants[1], new StatementConstraints())));
+        Assert.assertEquals(tail, getSet(temporal.queryInstantAfterInstant(instants[1], new StatementConstraints())));
+        temporal.close();
+    }
+
+    @Test
+    public void testGeoIndexing() throws Exception {
+        GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
+        Point p1 = gf.createPoint(new Coordinate(1, 1));
+        Point p2 = gf.createPoint(new Coordinate(2, 2));
+        GeoMesaGeoIndexer geo = new GeoMesaGeoIndexer();
+        geo.setConf(conf);
+        RyaStatement input = RyaStatement.builder()
+                .setSubject(new RyaURI(GRAPH + ":s"))
+                .setPredicate(new RyaURI(GRAPH + ":p"))
+                .setObject(new RyaType(GeoConstants.XMLSCHEMA_OGC_WKT, "Point(2 2)"))
+                .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, false);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, true);
+        RyaOutputFormat.setEntityEnabled(job, false);
+        write(input);
+        Set<Statement> expected = new HashSet<>();
+        Assert.assertEquals(expected, getSet(geo.queryContains(p1, new StatementConstraints())));
+        expected.add(RyaToRdfConversions.convertStatement(input));
+        Assert.assertEquals(expected, getSet(geo.queryEquals(p2, new StatementConstraints())));
+        geo.close();
+    }
+
+    @Test
+    public void testEntityIndexing() throws Exception {
+        EntityCentricIndex entity = new EntityCentricIndex();
+        entity.setConf(conf);
+        RyaStatement input = RyaStatement.builder()
+                .setSubject(new RyaURI(GRAPH + ":s"))
+                .setPredicate(new RyaURI(GRAPH + ":p"))
+                .setObject(new RyaURI(GRAPH + ":o"))
+                .build();
+        RyaOutputFormat.setCoreTablesEnabled(job, false);
+        RyaOutputFormat.setFreeTextEnabled(job, false);
+        RyaOutputFormat.setTemporalEnabled(job, false);
+        RyaOutputFormat.setGeoEnabled(job, false);
+        RyaOutputFormat.setEntityEnabled(job, true);
+        write(input);
+        entity.close();
+        Set<Statement> expected = new HashSet<>();
+        Set<Statement> inserted = new HashSet<>();
+        expected.add(RyaToRdfConversions.convertStatement(input));
+        String table = ConfigUtils.getEntityTableName(conf);
+        Scanner scanner = connector.createScanner(table, new Authorizations(CV));
+        for (Map.Entry<Key, Value> row : scanner) {
+            System.out.println(row);
+            inserted.add(RyaToRdfConversions.convertStatement(
+                    EntityCentricIndex.deserializeStatement(row.getKey(), row.getValue())));
+        }
+        Assert.assertEquals(expected, inserted);
+    }
+
+    private static <X> Set<X> getSet(CloseableIteration<X, ?> iter) throws Exception {
+        Set<X> set = new HashSet<X>();
+        while (iter.hasNext()) {
+            set.add(iter.next());
+        }
+        return set;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaStatementWritableTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaStatementWritableTest.java b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaStatementWritableTest.java
new file mode 100644
index 0000000..8bebdf4
--- /dev/null
+++ b/mapreduce/src/test/java/mvm/rya/accumulo/mr/RyaStatementWritableTest.java
@@ -0,0 +1,146 @@
+package mvm.rya.accumulo.mr;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.junit.Assert;
+import org.junit.Rule;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.domain.RyaType;
+import mvm.rya.api.domain.RyaURI;
+import mvm.rya.api.resolver.RyaTripleContext;
+
+/*
+ * 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.
+ */
+
+public class RyaStatementWritableTest {
+    private static final RyaURI s1 = new RyaURI(":s");
+    private static final RyaURI p1 = new RyaURI(":p");
+    private static final RyaType o1 = new RyaType(XMLSchema.INTEGER, "123");
+    private static final RyaURI s2 = new RyaURI(":s2");
+    private static final RyaURI p2 = new RyaURI(":p2");
+    private static final RyaType o2 = new RyaType(XMLSchema.STRING, "123");
+    private static final RyaURI graph1 = new RyaURI("http://example.org/graph1");
+    private static final RyaURI graph2 = new RyaURI("http://example.org/graph2");
+    private static final byte[] cv1 = "test_visibility".getBytes();
+    private static final long t1 = 1000;
+    private static final long t2 = 1001;
+    private static final RyaStatement rs1 = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph1).setColumnVisibility(cv1).setQualifier("q1").setTimestamp(t1).build();
+    // Equivalent:
+    private static final RyaStatement rs1b = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph1).setColumnVisibility(cv1).setQualifier("q1").setTimestamp(t1).build();
+    // Differ in one way each:
+    private static final RyaStatement rsGraph = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph2).setColumnVisibility(cv1).setQualifier("q1").setTimestamp(t1).build();
+    private static final RyaStatement rsCv = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph1).setColumnVisibility(null).setQualifier("q1").setTimestamp(t1).build();
+    private static final RyaStatement rsQualifier = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph1).setColumnVisibility(cv1).setQualifier("q2").setTimestamp(t1).build();
+    private static final RyaStatement rsTimestamp = RyaStatement.builder().setSubject(s1).setPredicate(p1).setObject(o1)
+            .setContext(graph1).setColumnVisibility(cv1).setQualifier("q1").setTimestamp(t2).build();
+    // Different triple:
+    private static final RyaStatement rs2 = RyaStatement.builder().setSubject(s2).setPredicate(p2).setObject(o2)
+            .setContext(graph1).setColumnVisibility(null).setQualifier("q1").setTimestamp(t1).build();
+
+    private static final RyaTripleContext ryaContext = RyaTripleContext.getInstance(new AccumuloRdfConfiguration());
+
+    @Rule
+    public ExpectedException expected = ExpectedException.none();
+
+    @Test
+    public void testEquals() throws Exception {
+        RyaStatementWritable rsw1 = new RyaStatementWritable(rs1, ryaContext);
+        RyaStatementWritable rsw1b = new RyaStatementWritable(rs1b, null);
+        RyaStatementWritable rsw2 = new RyaStatementWritable(rs2, ryaContext);
+        RyaStatementWritable rswNull = new RyaStatementWritable(null, ryaContext);
+        Assert.assertEquals("Equivalent statements should be equal", rsw1, rsw1b);
+        Assert.assertFalse("equals(null) should be false", rsw1.equals(null));
+        Assert.assertNotEquals("Statements representing different triples are not equal", rsw1, rsw2);
+        Assert.assertNotEquals("Statements representing different triples are not equal", rsw1, rswNull);
+        Assert.assertNotEquals("Statements with different named graphs are not equal", rsw1,
+                new RyaStatementWritable(rsGraph, ryaContext));
+        Assert.assertNotEquals("Statements with different column visibilities are not equal", rsw1,
+                new RyaStatementWritable(rsCv, ryaContext));
+        Assert.assertNotEquals("Statements with different column qualifiers are not equal", rsw1,
+                new RyaStatementWritable(rsQualifier, ryaContext));
+        Assert.assertNotEquals("Statements with different timestamps are not equal", rsw1,
+                new RyaStatementWritable(rsTimestamp, ryaContext));
+    }
+
+    @Test
+    public void testCompareTo() throws Exception {
+        RyaStatementWritable rsw1 = new RyaStatementWritable(rs1, ryaContext);
+        RyaStatementWritable rsw1b = new RyaStatementWritable(rs1b, null);
+        RyaStatementWritable rsw2 = new RyaStatementWritable(rs2, null);
+        RyaStatementWritable rswGraph = new RyaStatementWritable(rsCv, ryaContext);
+        RyaStatementWritable rswCv = new RyaStatementWritable(rsCv, ryaContext);
+        RyaStatementWritable rswQualifier = new RyaStatementWritable(rsQualifier, ryaContext);
+        RyaStatementWritable rswTimestamp = new RyaStatementWritable(rsTimestamp, ryaContext);
+        Assert.assertEquals("x.compareTo(x) should always return 0", 0, rsw1.compareTo(rsw1));
+        Assert.assertEquals("x.compareTo(x') where x and x' are equal should return 0", 0, rsw1.compareTo(rsw1b));
+        Assert.assertEquals("x.compareTo(x') where x and x' are equal should return 0", 0, rsw1b.compareTo(rsw1));
+        Assert.assertNotEquals("Statements with different named graphs are not equal", 0, rsw1.compareTo(rswGraph));
+        Assert.assertNotEquals("Statements with different column visibilities are not equal", 0, rsw1.compareTo(rswCv));
+        Assert.assertNotEquals("Statements with different column qualifiers are not equal", 0, rsw1.compareTo(rswQualifier));
+        Assert.assertNotEquals("Statements with different timestamps are not equal", 0, rsw1.compareTo(rswTimestamp));
+        Assert.assertEquals("compareTo in opposite directions should yield opposite signs",
+                Integer.signum(rsw1.compareTo(rsw2))*-1, Integer.signum(rsw2.compareTo(rsw1)));
+        // cycles shouldn't be possible; these comparisons can't all be negative or all be positive:
+        int x = Integer.signum(rsw1.compareTo(rsw2))
+                + Integer.signum(rsw2.compareTo(rsw1b))
+                + Integer.signum(rsw1b.compareTo(rsw1));
+        Assert.assertNotEquals("compareTo cycle detected", 3, Math.abs(x));
+        // compareTo(null) should always throw an exception:
+        expected.expect(NullPointerException.class);
+        rsw1.compareTo(null);
+    }
+
+    @Test
+    public void testSerializeDeserialize() throws Exception {
+        RyaStatementWritable rsw1 = new RyaStatementWritable(rs1, ryaContext);
+        RyaStatementWritable rsw2 = new RyaStatementWritable(rs2, ryaContext);
+        ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+        DataOutputStream bytesOut = new DataOutputStream(bytes);
+        rsw1.write(bytesOut);
+        rsw2.write(bytesOut);
+        DataInputStream bytesIn = new DataInputStream(new ByteArrayInputStream(bytes.toByteArray()));
+        RyaStatementWritable deserialized = new RyaStatementWritable();
+        // Verify initial deserialization:
+        deserialized.readFields(bytesIn);
+        Assert.assertEquals("Deserialized statement not equal to original", rsw1, deserialized);
+        Assert.assertEquals("Deserialized statement has different hash code", rsw1.hashCode(), deserialized.hashCode());
+        Assert.assertEquals("original.compareTo(deserialized) should equal 0", 0, rsw1.compareTo(deserialized));
+        // Verify that a second read mutates the Writable object into the correct second record:
+        RyaStatement deserializedStatement = deserialized.getRyaStatement();
+        deserialized.readFields(bytesIn);
+        Assert.assertEquals("Deserialized statement not overwritten on second read", rsw2, deserialized);
+        // Verify that the internal RyaStatement object is recreated, not overwritten:
+        RyaStatement deserializedStatement2 = deserialized.getRyaStatement();
+        Assert.assertNotSame("Reading a second record should create a new internal RyaStatement",
+                deserializedStatement, deserializedStatement2);
+    }
+}
\ No newline at end of file


[4/6] incubator-rya git commit: Consolidated MapReduce API and applications into toplevel project.

Posted by pu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/UpgradeObjectSerializationTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/UpgradeObjectSerializationTest.java b/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/UpgradeObjectSerializationTest.java
deleted file mode 100644
index b138292..0000000
--- a/dao/accumulo.rya/src/test/java/mvm/rya/accumulo/mr/upgrade/UpgradeObjectSerializationTest.java
+++ /dev/null
@@ -1,119 +0,0 @@
-package mvm.rya.accumulo.mr.upgrade;
-
-/*
- * 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 mvm.rya.api.resolver.impl.*;
-import org.junit.Test;
-
-import static mvm.rya.accumulo.mr.upgrade.Upgrade322Tool.UpgradeObjectSerialization;
-import static org.junit.Assert.*;
-
-public class UpgradeObjectSerializationTest {
-
-    @Test
-    public void testBooleanUpgrade() throws Exception {
-        String object = "true";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, BooleanRyaTypeResolver.BOOLEAN_LITERAL_MARKER);
-
-        assertEquals("1", upgrade);
-    }
-
-    @Test
-    public void testBooleanUpgradeFalse() throws Exception {
-        String object = "false";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, BooleanRyaTypeResolver.BOOLEAN_LITERAL_MARKER);
-
-        assertEquals("0", upgrade);
-    }
-
-    @Test
-    public void testByteUpgradeLowest() throws Exception {
-        String object = "-127";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, ByteRyaTypeResolver.LITERAL_MARKER);
-
-        assertEquals("81", upgrade);
-    }
-
-    @Test
-    public void testByteUpgradeHighest() throws Exception {
-        String object = "127";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, ByteRyaTypeResolver.LITERAL_MARKER);
-
-        assertEquals("7f", upgrade);
-    }
-
-    @Test
-    public void testLongUpgrade() throws Exception {
-        String object = "00000000000000000010";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, LongRyaTypeResolver.LONG_LITERAL_MARKER);
-
-        assertEquals("800000000000000a", upgrade);
-    }
-
-    @Test
-    public void testIntUpgrade() throws Exception {
-        String object = "00000000010";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, IntegerRyaTypeResolver.INTEGER_LITERAL_MARKER);
-
-        assertEquals("8000000a", upgrade);
-    }
-
-    @Test
-    public void testDateTimeUpgrade() throws Exception {
-        String object = "9223370726404375807";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, DateTimeRyaTypeResolver.DATETIME_LITERAL_MARKER);
-
-        assertEquals("800001311cee3b00", upgrade);
-    }
-
-    @Test
-    public void testDoubleUpgrade() throws Exception {
-        String object = "00001 1.0";
-        final UpgradeObjectSerialization upgradeObjectSerialization
-          = new UpgradeObjectSerialization();
-        final String upgrade = upgradeObjectSerialization
-          .upgrade(object, DoubleRyaTypeResolver.DOUBLE_LITERAL_MARKER);
-
-        assertEquals("c024000000000000", upgrade);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/resources/namedgraphs.trig
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/resources/namedgraphs.trig b/dao/accumulo.rya/src/test/resources/namedgraphs.trig
deleted file mode 100644
index b647632..0000000
--- a/dao/accumulo.rya/src/test/resources/namedgraphs.trig
+++ /dev/null
@@ -1,7 +0,0 @@
-@prefix rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#> .
-@prefix xsd: <http://www.w3.org/2001/XMLSchema#> .
-@prefix swp: <http://www.w3.org/2004/03/trix/swp-1/> .
-@prefix dc: <http://purl.org/dc/elements/1.1/> .
-@prefix ex: <http://www.example.org/vocabulary#> .
-@prefix : <http://www.example.org/exampleDocument#> .
-:G1 { :Monica ex:name "Monica Murphy" . }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/dao/accumulo.rya/src/test/resources/test.ntriples
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/resources/test.ntriples b/dao/accumulo.rya/src/test/resources/test.ntriples
deleted file mode 100644
index 26a0a17..0000000
--- a/dao/accumulo.rya/src/test/resources/test.ntriples
+++ /dev/null
@@ -1 +0,0 @@
-<urn:lubm:rdfts#GraduateStudent01> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent02> .
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullFreeTextIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullFreeTextIndexer.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullFreeTextIndexer.java
deleted file mode 100644
index 147050f..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullFreeTextIndexer.java
+++ /dev/null
@@ -1,102 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 info.aduna.iteration.CloseableIteration;
-
-import java.io.IOException;
-import java.util.Set;
-
-import mvm.rya.accumulo.experimental.AbstractAccumuloIndexer;
-import mvm.rya.api.RdfCloudTripleStoreConfiguration;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.indexing.FreeTextIndexer;
-import mvm.rya.indexing.StatementConstraints;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.hadoop.conf.Configuration;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.query.QueryEvaluationException;
-
-public class NullFreeTextIndexer extends AbstractAccumuloIndexer implements FreeTextIndexer {
-
-    @Override
-    public String getTableName() {
-        return null;
-    }
-
-    @Override
-    public void storeStatement(RyaStatement statement) throws IOException {
-    }
-
-    @Override
-    public Configuration getConf() {
-        return null;
-    }
-
-    @Override
-    public void setConf(Configuration arg0) {
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryText(String query, StatementConstraints contraints)
-            throws IOException {
-        return null;
-    }
-
-    @Override
-    public Set<URI> getIndexablePredicates() {
-        return null;
-    }
-
-	@Override
-	public void init() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void setConnector(Connector connector) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void destroy() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void purge(RdfCloudTripleStoreConfiguration configuration) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void dropAndDestroy() {
-		// TODO Auto-generated method stub
-		
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullGeoIndexer.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullGeoIndexer.java
deleted file mode 100644
index fe26f6f..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullGeoIndexer.java
+++ /dev/null
@@ -1,153 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 info.aduna.iteration.CloseableIteration;
-
-import java.io.IOException;
-import java.util.Set;
-
-import mvm.rya.accumulo.experimental.AbstractAccumuloIndexer;
-import mvm.rya.api.RdfCloudTripleStoreConfiguration;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.StatementConstraints;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.hadoop.conf.Configuration;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.vividsolutions.jts.geom.Geometry;
-
-public class NullGeoIndexer extends AbstractAccumuloIndexer implements GeoIndexer {
-
-    @Override
-    public String getTableName() {
-
-        return null;
-    }
-
-    @Override
-    public void storeStatement(RyaStatement statement) throws IOException {
-
-        
-    }
-
-    @Override
-    public Configuration getConf() {
-
-        return null;
-    }
-
-    @Override
-    public void setConf(Configuration arg0) {
-
-        
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryEquals(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryTouches(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryWithin(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryContains(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(Geometry query, StatementConstraints contraints) {
-
-        return null;
-    }
-
-    @Override
-    public Set<URI> getIndexablePredicates() {
-
-        return null;
-    }
-
-	@Override
-	public void init() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void setConnector(Connector connector) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void destroy() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void purge(RdfCloudTripleStoreConfiguration configuration) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void dropAndDestroy() {
-		// TODO Auto-generated method stub
-		
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullTemporalIndexer.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullTemporalIndexer.java
deleted file mode 100644
index cbe36b8..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/NullTemporalIndexer.java
+++ /dev/null
@@ -1,186 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 info.aduna.iteration.CloseableIteration;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Set;
-
-import mvm.rya.accumulo.experimental.AbstractAccumuloIndexer;
-import mvm.rya.api.RdfCloudTripleStoreConfiguration;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.indexing.StatementConstraints;
-import mvm.rya.indexing.TemporalIndexer;
-import mvm.rya.indexing.TemporalInstant;
-import mvm.rya.indexing.TemporalInterval;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.hadoop.conf.Configuration;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.query.QueryEvaluationException;
-
-/**
- * Temporal Indexer that does nothing, like when disabled.
- *
- */
-public class NullTemporalIndexer extends AbstractAccumuloIndexer implements TemporalIndexer {
-
-    @Override
-    public String getTableName() {
-
-        return null;
-    }
-
-    @Override
-    public void storeStatement(RyaStatement statement) throws IOException {
-
-        
-    }
-
-    @Override
-    public Configuration getConf() {
-
-        return null;
-    }
-
-    @Override
-    public void setConf(Configuration arg0) {
-
-        
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantEqualsInstant(TemporalInstant queryInstant,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantBeforeInstant(TemporalInstant queryInstant,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantAfterInstant(TemporalInstant queryInstant,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantBeforeInterval(TemporalInterval givenInterval,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantAfterInterval(TemporalInterval givenInterval,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantInsideInterval(TemporalInterval givenInterval,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantHasBeginningInterval(TemporalInterval queryInterval,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryInstantHasEndInterval(TemporalInterval queryInterval,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntervalEquals(TemporalInterval query,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntervalBefore(TemporalInterval query,
-            StatementConstraints contraints) throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntervalAfter(TemporalInterval query, StatementConstraints contraints)
-            throws QueryEvaluationException {
-
-        return null;
-    }
-
-    @Override
-    public Set<URI> getIndexablePredicates() {
-
-        return null;
-    }
-
-	@Override
-	public void init() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void setConnector(Connector connector) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void destroy() {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void purge(RdfCloudTripleStoreConfiguration configuration) {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void dropAndDestroy() {
-		// TODO Auto-generated method stub
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
deleted file mode 100644
index 7e690f4..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/RyaOutputFormat.java
+++ /dev/null
@@ -1,329 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 java.io.Closeable;
-import java.io.Flushable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.AccumuloRyaDAO;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.persist.RyaDAOException;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import mvm.rya.indexing.FreeTextIndexer;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.StatementSerializer;
-import mvm.rya.indexing.TemporalIndexer;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
-import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
-import mvm.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
-
-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.Mutation;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.log4j.Logger;
-import org.geotools.feature.SchemaException;
-import org.openrdf.model.Statement;
-
-/**
- * Hadoop Map/Reduce class to use Rya, the {@link GeoIndexer}, the {@link FreeTextIndexer}, and the {@link TemporalIndexer} as the sink of {@link Statement} data.
- * wrapped in an {@link StatementWritable} objects. This {@link OutputFormat} ignores the Keys and only writes the Values to Rya.
- * 
- * The user must specify connection parameters for Rya, {@link GeoIndexer}, {@link FreeTextIndexer}, and {@link TemporalIndexer}.
- */
-public class RyaOutputFormat extends OutputFormat<Writable, StatementWritable> {
-    private static final Logger logger = Logger.getLogger(RyaOutputFormat.class);
-
-    private static final String PREFIX = RyaOutputFormat.class.getSimpleName();
-    private static final String MAX_MUTATION_BUFFER_SIZE = PREFIX + ".maxmemory";
-    private static final String ENABLE_FREETEXT = PREFIX + ".freetext.enable";
-    private static final String ENABLE_GEO = PREFIX + ".geo.enable";
-    private static final String ENABLE_TEMPORAL = PREFIX + ".temporal.enable";;
-
-
-    @Override
-    public void checkOutputSpecs(JobContext jobContext) throws IOException, InterruptedException {
-        Configuration conf = jobContext.getConfiguration();
-
-        // make sure that all of the indexers can connect
-        getGeoIndexer(conf);
-        getFreeTextIndexer(conf);
-        getTemporalIndexer(conf);
-        getRyaIndexer(conf);
-    }
-
-    @Override
-    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException {
-        // copied from AccumuloOutputFormat
-        return new NullOutputFormat<Text, Mutation>().getOutputCommitter(context);
-    }
-
-    @Override
-    public RecordWriter<Writable, StatementWritable> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-        return new RyaRecordWriter(context);
-    }
-
-    private static GeoIndexer getGeoIndexer(Configuration conf) throws IOException {
-        if (!conf.getBoolean(ENABLE_GEO, true)) {
-            return new NullGeoIndexer();
-        }
-
-        GeoMesaGeoIndexer geo = new GeoMesaGeoIndexer();
-        geo.setConf(conf);
-        return geo;
-
-    }
-
-    private static FreeTextIndexer getFreeTextIndexer(Configuration conf) throws IOException {
-        if (!conf.getBoolean(ENABLE_FREETEXT, true)) {
-            return new NullFreeTextIndexer();
-        }
-
-        AccumuloFreeTextIndexer freeText = new AccumuloFreeTextIndexer();
-        freeText.setConf(conf);
-        return freeText;
-
-    }
-
-    private static TemporalIndexer getTemporalIndexer(Configuration conf) throws IOException {
-        if (!conf.getBoolean(ENABLE_TEMPORAL, true)) {
-            return new NullTemporalIndexer();
-        }
-        AccumuloTemporalIndexer temporal = new AccumuloTemporalIndexer();
-        temporal.setConf(conf);
-        return temporal;
-    }
-
-    private static AccumuloRyaDAO getRyaIndexer(Configuration conf) throws IOException {
-        try {
-            AccumuloRyaDAO ryaIndexer = new AccumuloRyaDAO();
-            Connector conn = ConfigUtils.getConnector(conf);
-            ryaIndexer.setConnector(conn);
-
-            AccumuloRdfConfiguration ryaConf = new AccumuloRdfConfiguration();
-
-            String tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, null);
-            if (tablePrefix != null) {
-                ryaConf.setTablePrefix(tablePrefix);
-            }
-            ryaConf.setDisplayQueryPlan(false);
-            ryaIndexer.setConf(ryaConf);
-            ryaIndexer.init();
-            return ryaIndexer;
-        } catch (AccumuloException e) {
-            logger.error("Cannot create RyaIndexer", e);
-            throw new IOException(e);
-        } catch (AccumuloSecurityException e) {
-            logger.error("Cannot create RyaIndexer", e);
-            throw new IOException(e);
-        } catch (RyaDAOException e) {
-            logger.error("Cannot create RyaIndexer", e);
-            throw new IOException(e);
-        }
-    }
-
-    public static class RyaRecordWriter extends RecordWriter<Writable, StatementWritable> implements Closeable, Flushable {
-        private static final Logger logger = Logger.getLogger(RyaRecordWriter.class);
-
-        private FreeTextIndexer freeTextIndexer;
-        private GeoIndexer geoIndexer;
-        private TemporalIndexer temporalIndexer;
-        private AccumuloRyaDAO ryaIndexer;
-
-        private static final long ONE_MEGABYTE = 1024L * 1024L;
-        private static final long AVE_STATEMENT_SIZE = 100L;
-
-        private long bufferSizeLimit;
-        private long bufferCurrentSize = 0;
-
-        private ArrayList<RyaStatement> buffer;
-
-        public RyaRecordWriter(TaskAttemptContext context) throws IOException {
-            this(context.getConfiguration());
-        }
-
-        public RyaRecordWriter(Configuration conf) throws IOException {
-            // set up the buffer
-            bufferSizeLimit = conf.getLong(MAX_MUTATION_BUFFER_SIZE, ONE_MEGABYTE);
-            int bufferCapacity = (int) (bufferSizeLimit / AVE_STATEMENT_SIZE);
-            buffer = new ArrayList<RyaStatement>(bufferCapacity);
-
-            // set up the indexers
-            freeTextIndexer = getFreeTextIndexer(conf);
-            geoIndexer = getGeoIndexer(conf);
-            temporalIndexer = getTemporalIndexer(conf);
-            ryaIndexer = getRyaIndexer(conf);
-
-            // update fields used for metrics
-            startTime = System.currentTimeMillis();
-            lastCommitFinishTime = startTime;
-        }
-
-        @Override
-        public void flush() throws IOException {
-            flushBuffer();
-        }
-
-        @Override
-        public void close() throws IOException {
-            close(null);
-        }
-
-        @Override
-        public void close(TaskAttemptContext paramTaskAttemptContext) throws IOException {
-            // close everything. log errors
-            try {
-                flush();
-            } catch (IOException e) {
-                logger.error("Error flushing the buffer on RyaOutputFormat Close", e);
-            }
-            try {
-                if (geoIndexer != null)
-                    geoIndexer.close();
-            } catch (IOException e) {
-                logger.error("Error closing the geoIndexer on RyaOutputFormat Close", e);
-            }
-            try {
-                if (freeTextIndexer != null)
-                    freeTextIndexer.close();
-            } catch (IOException e) {
-                logger.error("Error closing the freetextIndexer on RyaOutputFormat Close", e);
-            }
-            try {
-                if (temporalIndexer != null)
-                    temporalIndexer.close();
-            } catch (IOException e) {
-                logger.error("Error closing the temporalIndexer on RyaOutputFormat Close", e);
-            }
-            try {
-                ryaIndexer.destroy();
-            } catch (RyaDAOException e) {
-                logger.error("Error closing RyaDAO on RyaOutputFormat Close", e);
-            }
-        }
-
-        public void write(Statement statement) throws IOException, InterruptedException {
-            write(null, new StatementWritable(statement));
-        }
-
-        @Override
-        public void write(Writable key, StatementWritable value) throws IOException, InterruptedException {
-            buffer.add(RdfToRyaConversions.convertStatement(value));
-
-            bufferCurrentSize += StatementSerializer.writeStatement(value).length();
-
-            if (bufferCurrentSize >= bufferSizeLimit) {
-                flushBuffer();
-            }
-        }
-
-        // fields for storing metrics
-        private long startTime = 0;
-        private long lastCommitFinishTime = 0;
-        private long totalCommitRecords = 0;
-
-        private double totalReadDuration = 0;
-        private double totalWriteDuration = 0;
-
-        private long commitCount = 0;
-
-        private void flushBuffer() throws IOException {
-            totalCommitRecords += buffer.size();
-            commitCount++;
-
-            long startCommitTime = System.currentTimeMillis();
-
-            logger.info(String.format("(C-%d) Flushing buffer with %,d objects and %,d bytes", commitCount, buffer.size(),
-                    bufferCurrentSize));
-
-            double readingDuration = (startCommitTime - lastCommitFinishTime) / 1000.;
-            totalReadDuration += readingDuration;
-            double currentReadRate = buffer.size() / readingDuration;
-            double totalReadRate = totalCommitRecords / totalReadDuration;
-
-            // Print "reading" metrics
-            logger.info(String.format("(C-%d) (Reading) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, readingDuration,
-                    currentReadRate, totalReadRate));
-
-            // write to geo
-            geoIndexer.storeStatements(buffer);
-            geoIndexer.flush();
-
-            // write to free text
-            freeTextIndexer.storeStatements(buffer);
-            freeTextIndexer.flush();
-
-            // write to temporal
-            temporalIndexer.storeStatements(buffer);
-            temporalIndexer.flush();
-
-            // write to rya
-            try {
-                ryaIndexer.add(buffer.iterator());
-            } catch (RyaDAOException e) {
-                logger.error("Cannot writing statement to Rya", e);
-                throw new IOException(e);
-            }
-
-            lastCommitFinishTime = System.currentTimeMillis();
-
-            double writingDuration = (lastCommitFinishTime - startCommitTime) / 1000.;
-            totalWriteDuration += writingDuration;
-            double currentWriteRate = buffer.size() / writingDuration;
-            double totalWriteRate = totalCommitRecords / totalWriteDuration;
-
-            // Print "writing" stats
-            logger.info(String.format("(C-%d) (Writing) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, writingDuration,
-                    currentWriteRate, totalWriteRate));
-
-            double processDuration = writingDuration + readingDuration;
-            double totalProcessDuration = totalWriteDuration + totalReadDuration;
-            double currentProcessRate = buffer.size() / processDuration;
-            double totalProcessRate = totalCommitRecords / (totalProcessDuration);
-
-            // Print "total" stats
-            logger.info(String.format("(C-%d) (Total) Duration, Current Rate, Total Rate: %.2f %.2f %.2f ", commitCount, processDuration,
-                    currentProcessRate, totalProcessRate));
-
-            // clear the buffer
-            buffer.clear();
-            bufferCurrentSize = 0L;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/StatementWritable.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/StatementWritable.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/StatementWritable.java
deleted file mode 100644
index 629baf2..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/StatementWritable.java
+++ /dev/null
@@ -1,86 +0,0 @@
-package mvm.rya.accumulo.mr;
-
-/*
- * 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 java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-
-import mvm.rya.indexing.StatementSerializer;
-
-/**
- * A {@link Writable} wrapper for {@link Statement} objects.
- */
-@SuppressWarnings("serial")
-public class StatementWritable implements Statement, Writable {
-
-    private Statement statement;
-
-    public StatementWritable(Statement statement) {
-        setStatement(statement);
-    }
-
-    public void setStatement(Statement statement) {
-        this.statement = statement;
-    }
-
-    public Statement getStatement() {
-        return statement;
-    }
-
-    @Override
-    public void readFields(DataInput paramDataInput) throws IOException {
-        statement = StatementSerializer.readStatement(paramDataInput.readUTF());
-    }
-
-    @Override
-    public void write(DataOutput paramDataOutput) throws IOException {
-        paramDataOutput.writeUTF(StatementSerializer.writeStatement(statement));
-    }
-
-    @Override
-    public Resource getSubject() {
-        return statement.getSubject();
-    }
-
-    @Override
-    public URI getPredicate() {
-        return statement.getPredicate();
-    }
-
-    @Override
-    public Value getObject() {
-        return statement.getObject();
-    }
-
-    @Override
-    public Resource getContext() {
-        return statement.getContext();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputToolIndexing.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputToolIndexing.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputToolIndexing.java
deleted file mode 100644
index ecc2354..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/BulkNtripsInputToolIndexing.java
+++ /dev/null
@@ -1,227 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.io.StringReader;
-
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-import mvm.rya.indexing.FreeTextIndexer;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
-import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Logger;
-import org.geotools.feature.SchemaException;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ContextStatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.rio.ParserConfig;
-import org.openrdf.rio.RDFFormat;
-import org.openrdf.rio.RDFHandlerException;
-import org.openrdf.rio.RDFParseException;
-import org.openrdf.rio.RDFParser;
-import org.openrdf.rio.Rio;
-import org.openrdf.rio.helpers.RDFHandlerBase;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Take large ntrips files and use MapReduce to ingest into other indexing
- */
-public class BulkNtripsInputToolIndexing extends Configured implements Tool {
-
-    private String userName = null;
-    private String pwd = null;
-    private String instance = null;
-    private String zk = null;
-
-    private String format = RDFFormat.NTRIPLES.getName();
-
-    @Override
-    public int run(final String[] args) throws Exception {
-        final Configuration conf = getConf();
-        // conf
-        zk = conf.get(MRUtils.AC_ZK_PROP, zk);
-        instance = conf.get(MRUtils.AC_INSTANCE_PROP, instance);
-        userName = conf.get(MRUtils.AC_USERNAME_PROP, userName);
-        pwd = conf.get(MRUtils.AC_PWD_PROP, pwd);
-        format = conf.get(MRUtils.FORMAT_PROP, format);
-
-        String auths = conf.get(MRUtils.AC_CV_PROP, "");
-
-        conf.set(MRUtils.FORMAT_PROP, format);
-        Preconditions.checkNotNull(zk, MRUtils.AC_ZK_PROP + " not set");
-        Preconditions.checkNotNull(instance, MRUtils.AC_INSTANCE_PROP + " not set");
-        Preconditions.checkNotNull(userName, MRUtils.AC_USERNAME_PROP + " not set");
-        Preconditions.checkNotNull(pwd, MRUtils.AC_PWD_PROP + " not set");
-
-        // map the config values to free text configu values
-        conf.set(ConfigUtils.CLOUDBASE_ZOOKEEPERS, zk);
-        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, instance);
-        conf.set(ConfigUtils.CLOUDBASE_USER, userName);
-        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, pwd);
-        conf.set(ConfigUtils.CLOUDBASE_AUTHS, auths);
-
-        final String inputDir = args[0];
-
-        String tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, null);
-        Preconditions.checkNotNull(tablePrefix, MRUtils.TABLE_PREFIX_PROPERTY + " not set");
-
-        String docTextTable = tablePrefix + "text";
-        conf.set(ConfigUtils.FREE_TEXT_DOC_TABLENAME, docTextTable);
-
-        String docTermTable = tablePrefix + "terms";
-        conf.set(ConfigUtils.FREE_TEXT_TERM_TABLENAME, docTermTable);
-
-        String geoTable = tablePrefix + "geo";
-        conf.set(ConfigUtils.GEO_TABLENAME, geoTable);
-
-        System.out.println("Loading data into tables[freetext, geo]");
-        System.out.println("Loading data into tables[" + docTermTable + " " + docTextTable + " " + geoTable + "]");
-
-        Job job = new Job(new Configuration(conf), "Bulk Ingest load data into Indexing Tables");
-        job.setJarByClass(this.getClass());
-
-        // setting long job
-        Configuration jobConf = job.getConfiguration();
-        jobConf.setBoolean("mapred.map.tasks.speculative.execution", false);
-        jobConf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
-        jobConf.set("io.sort.mb", jobConf.get("io.sort.mb", "256"));
-        jobConf.setBoolean("mapred.compress.map.output", true);
-
-        job.setInputFormatClass(TextInputFormat.class);
-
-        job.setMapperClass(ParseNtripsMapper.class);
-
-        // I'm not actually going to write output.
-        job.setOutputFormatClass(NullOutputFormat.class);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(Text.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(Text.class);
-
-        TextInputFormat.setInputPaths(job, new Path(inputDir));
-
-        job.setNumReduceTasks(0);
-
-        job.waitForCompletion(true);
-
-        return 0;
-    }
-
-    public static void main(String[] args) throws Exception {
-        ToolRunner.run(new Configuration(), new BulkNtripsInputToolIndexing(), args);
-    }
-
-    public static class ParseNtripsMapper extends Mapper<LongWritable, Text, Text, Text> {
-        private static final Logger logger = Logger.getLogger(ParseNtripsMapper.class);
-
-        public static final String TABLE_PROPERTY = "parsentripsmapper.table";
-
-        private RDFParser parser;
-        private FreeTextIndexer freeTextIndexer;
-        private GeoIndexer geoIndexer;
-        private String rdfFormat;
-
-        @Override
-        protected void setup(final Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-
-            freeTextIndexer = new AccumuloFreeTextIndexer();
-            freeTextIndexer.setConf(conf);
-            geoIndexer = new GeoMesaGeoIndexer();
-            geoIndexer.setConf(conf);
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            rdfFormat = conf.get(MRUtils.FORMAT_PROP);
-            checkNotNull(rdfFormat, "Rdf format cannot be null");
-
-            String namedGraphString = conf.get(MRUtils.NAMED_GRAPH_PROP);
-            checkNotNull(namedGraphString, MRUtils.NAMED_GRAPH_PROP + " cannot be null");
-
-            final Resource namedGraph = vf.createURI(namedGraphString);
-
-            parser = Rio.createParser(RDFFormat.valueOf(rdfFormat));
-            parser.setParserConfig(new ParserConfig(true, true, true, RDFParser.DatatypeHandling.VERIFY));
-            parser.setRDFHandler(new RDFHandlerBase() {
-
-                @Override
-                public void handleStatement(Statement statement) throws RDFHandlerException {
-                    Statement contextStatement = new ContextStatementImpl(statement.getSubject(), statement
-                            .getPredicate(), statement.getObject(), namedGraph);
-                    try {
-                        freeTextIndexer.storeStatement(RdfToRyaConversions.convertStatement(contextStatement));
-                        geoIndexer.storeStatement(RdfToRyaConversions.convertStatement(contextStatement));
-                    } catch (IOException e) {
-                        logger.error("Error creating indexers", e);
-                    }
-                }
-            });
-        }
-
-        @Override
-        public void map(LongWritable key, Text value, Context output) throws IOException, InterruptedException {
-            String rdf = value.toString();
-            try {
-                parser.parse(new StringReader(rdf), "");
-            } catch (RDFParseException e) {
-                System.out.println("Line[" + rdf + "] cannot be formatted with format[" + rdfFormat + "]. Exception[" + e.getMessage()
-                        + "]");
-            } catch (Exception e) {
-                logger.error("error during map", e);
-                throw new IOException("Exception occurred parsing triple[" + rdf + "]");
-            }
-        }
-
-        @Override
-        public void cleanup(Context context) {
-            IOUtils.closeStream(freeTextIndexer);
-            IOUtils.closeStream(geoIndexer);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/RyaBatchWriterInputTool.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/RyaBatchWriterInputTool.java b/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/RyaBatchWriterInputTool.java
deleted file mode 100644
index fb80804..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/mr/fileinput/RyaBatchWriterInputTool.java
+++ /dev/null
@@ -1,243 +0,0 @@
-package mvm.rya.accumulo.mr.fileinput;
-
-/*
- * 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 static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.io.StringReader;
-
-import mvm.rya.accumulo.mr.RyaOutputFormat;
-import mvm.rya.accumulo.mr.StatementWritable;
-import mvm.rya.accumulo.mr.utils.MRUtils;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-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.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Logger;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.rio.ParserConfig;
-import org.openrdf.rio.RDFFormat;
-import org.openrdf.rio.RDFHandlerException;
-import org.openrdf.rio.RDFParseException;
-import org.openrdf.rio.RDFParser;
-import org.openrdf.rio.Rio;
-import org.openrdf.rio.helpers.RDFHandlerBase;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Take large ntrips files and use MapReduce to ingest into other indexing
- */
-public class RyaBatchWriterInputTool extends Configured implements Tool {
-    private static final Logger logger = Logger.getLogger(RyaBatchWriterInputTool.class);
-
-    @Override
-    public int run(final String[] args) throws Exception {
-        String userName = null;
-        String pwd = null;
-        String instance = null;
-        String zk = null;
-        String format = null;
-
-        final Configuration conf = getConf();
-        // conf
-        zk = conf.get(MRUtils.AC_ZK_PROP, zk);
-        instance = conf.get(MRUtils.AC_INSTANCE_PROP, instance);
-        userName = conf.get(MRUtils.AC_USERNAME_PROP, userName);
-        pwd = conf.get(MRUtils.AC_PWD_PROP, pwd);
-        format = conf.get(MRUtils.FORMAT_PROP,  RDFFormat.NTRIPLES.getName());
-
-        String auths = conf.get(MRUtils.AC_CV_PROP, "");
-
-        conf.set(MRUtils.FORMAT_PROP, format);
-        Preconditions.checkNotNull(zk, MRUtils.AC_ZK_PROP + " not set");
-        Preconditions.checkNotNull(instance, MRUtils.AC_INSTANCE_PROP + " not set");
-        Preconditions.checkNotNull(userName, MRUtils.AC_USERNAME_PROP + " not set");
-        Preconditions.checkNotNull(pwd, MRUtils.AC_PWD_PROP + " not set");
-
-        // map the config values to free text configure values
-        conf.set(ConfigUtils.CLOUDBASE_ZOOKEEPERS, zk);
-        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, instance);
-        conf.set(ConfigUtils.CLOUDBASE_USER, userName);
-        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, pwd);
-        conf.set(ConfigUtils.CLOUDBASE_AUTHS, auths);
-
-        final String inputDir = args[0];
-
-        String tablePrefix = conf.get(MRUtils.TABLE_PREFIX_PROPERTY, null);
-        Preconditions.checkNotNull(tablePrefix, MRUtils.TABLE_PREFIX_PROPERTY + " not set");
-
-        String docTextTable = tablePrefix + "text";
-        conf.set(ConfigUtils.FREE_TEXT_DOC_TABLENAME, docTextTable);
-
-        String docTermTable = tablePrefix + "terms";
-        conf.set(ConfigUtils.FREE_TEXT_TERM_TABLENAME, docTermTable);
-
-        String geoTable = tablePrefix + "geo";
-        conf.set(ConfigUtils.GEO_TABLENAME, geoTable);
-
-        logger.info("Loading data into tables[rya, freetext, geo]");
-        logger.info("Loading data into tables[" + docTermTable + " " + docTextTable + " " + geoTable + "]");
-
-        Job job = new Job(new Configuration(conf), "Batch Writer load data into Rya Core and Indexing Tables");
-        job.setJarByClass(this.getClass());
-
-        // setting long job
-        Configuration jobConf = job.getConfiguration();
-        jobConf.setBoolean("mapred.map.tasks.speculative.execution", false);
-
-        jobConf.setInt("mapred.task.timeout", 1000 * 60 * 60 * 24); // timeout after 1 day
-
-        job.setInputFormatClass(TextInputFormat.class);
-
-        job.setMapperClass(ParseNtripsMapper.class);
-
-        job.setNumReduceTasks(0);
-        
-        // Use Rya Output Format
-        job.setOutputFormatClass(RyaOutputFormat.class);
-        job.setOutputKeyClass(NullWritable.class);
-        job.setOutputValueClass(StatementWritable.class);
-        job.setMapOutputKeyClass(NullWritable.class);
-        job.setMapOutputValueClass(StatementWritable.class);
-
-        TextInputFormat.setInputPaths(job, new Path(inputDir));
-
-        job.waitForCompletion(true);
-
-        return 0;
-    }
-
-    public static void main(String[] args) throws Exception {
-        ToolRunner.run(new Configuration(), new RyaBatchWriterInputTool(), args);
-    }
-
-    public static class ParseNtripsMapper extends Mapper<LongWritable, Text, Writable, Statement> {
-        private static final Logger logger = Logger.getLogger(ParseNtripsMapper.class);
-
-        private RDFParser parser;
-        private RDFFormat rdfFormat;
-
-        @Override
-        protected void setup(final Context context) throws IOException, InterruptedException {
-            super.setup(context);
-            Configuration conf = context.getConfiguration();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            String rdfFormatName = conf.get(MRUtils.FORMAT_PROP);
-            checkNotNull(rdfFormatName, "Rdf format cannot be null");
-            rdfFormat = RDFFormat.valueOf(rdfFormatName);
-
-            String namedGraphString = conf.get(MRUtils.NAMED_GRAPH_PROP);
-            checkNotNull(namedGraphString, MRUtils.NAMED_GRAPH_PROP + " cannot be null");
-
-            final Resource namedGraph = vf.createURI(namedGraphString);
-
-            parser = Rio.createParser(rdfFormat);
-            parser.setParserConfig(new ParserConfig(true, true, true, RDFParser.DatatypeHandling.VERIFY));
-            parser.setRDFHandler(new RDFHandlerBase() {
-                @Override
-                public void handleStatement(Statement statement) throws RDFHandlerException {
-                    Statement output;
-                    if (rdfFormat.equals(RDFFormat.NTRIPLES)) {
-                        output = new ConextStatementWrapper(statement, namedGraph);
-                    } else {
-                        output = statement;
-                    }
-                    try {
-                        context.write(NullWritable.get(), new StatementWritable(output));
-                    } catch (IOException e) {
-                        logger.error("Error writing statement", e);
-                        throw new RDFHandlerException(e);
-                    } catch (InterruptedException e) {
-                        logger.error("Error writing statement", e);
-                        throw new RDFHandlerException(e);
-                    }
-                }
-
-            });
-        }
-
-        @Override
-        public void map(LongWritable key, Text value, Context output) throws IOException, InterruptedException {
-            String rdf = value.toString();
-            try {
-                parser.parse(new StringReader(rdf), "");
-            } catch (RDFParseException e) {
-                logger.error("Line[" + rdf + "] cannot be formatted with format[" + rdfFormat + "]. Exception[" + e.getMessage()
-                        + "]", e);
-            } catch (Exception e) {
-                logger.error("error during map", e);
-                throw new IOException("Exception occurred parsing triple[" + rdf + "]", e);
-            }
-        }
-    }
-
-    @SuppressWarnings("serial")
-    private static class ConextStatementWrapper implements Statement {
-        private Statement statementWithoutConext;
-        private Resource context;
-
-        public ConextStatementWrapper(Statement statementWithoutConext, Resource context) {
-            this.statementWithoutConext = statementWithoutConext;
-            this.context = context;
-        }
-
-        @Override
-        public Resource getSubject() {
-            return statementWithoutConext.getSubject();
-        }
-
-        @Override
-        public URI getPredicate() {
-            return statementWithoutConext.getPredicate();
-        }
-
-        @Override
-        public Value getObject() {
-            return statementWithoutConext.getObject();
-        }
-
-        @Override
-        public Resource getContext() {
-            return context;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/markdown/_index.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/_index.md b/extras/rya.manual/src/site/markdown/_index.md
index bf030a3..6e45779 100644
--- a/extras/rya.manual/src/site/markdown/_index.md
+++ b/extras/rya.manual/src/site/markdown/_index.md
@@ -28,6 +28,7 @@
 - [Evaluation Table](eval.md)
 - [Pre-computed Joins](loadPrecomputedJoin.md)
 - [Inferencing](infer.md)
+- [MapReduce Interface](mapreduce.md)
 
 # Samples
 - [Typical First Steps](sm-firststeps.md)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/index.md b/extras/rya.manual/src/site/markdown/index.md
index 0748284..2840d10 100644
--- a/extras/rya.manual/src/site/markdown/index.md
+++ b/extras/rya.manual/src/site/markdown/index.md
@@ -30,6 +30,7 @@ This project contains documentation about the Rya, a scalable RDF triple store o
 - [Evaluation Table](eval.md)
 - [Pre-computed Joins](loadPrecomputedJoin.md)
 - [Inferencing](infer.md)
+- [MapReduce Interface](mapreduce.md)
 
 # Samples
 - [Typical First Steps](sm-firststeps.md)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/markdown/loaddata.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/loaddata.md b/extras/rya.manual/src/site/markdown/loaddata.md
index 2c6bc00..74fb90b 100644
--- a/extras/rya.manual/src/site/markdown/loaddata.md
+++ b/extras/rya.manual/src/site/markdown/loaddata.md
@@ -96,10 +96,10 @@ Bulk loading data is done through Map Reduce jobs
 
 ### Bulk Load RDF data
 
-This Map Reduce job will read a full file into memory and parse it into statements. The statements are saved into the store. Here is an example for storing in Accumulo:
+This Map Reduce job will read files into memory and parse them into statements. The statements are saved into the store. Here is an example for storing in Accumulo:
 
 ```
-hadoop jar target/accumulo.rya-3.0.4-SNAPSHOT-shaded.jar mvm.rya.accumulo.mr.fileinput.BulkNtripsInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=triplestore_ -Dio.sort.mb=64 /tmp/temp.ntrips
+hadoop jar target/rya.mapreduce-3.2.10-SNAPSHOT-shaded.jar mvm.rya.accumulo.mr.RdfFileInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=triplestore_ -Drdf.format=N-Triples /tmp/temp.ntrips
 ```
 
 Options:
@@ -107,9 +107,14 @@ Options:
 - rdf.tablePrefix : The tables (spo, po, osp) are prefixed with this qualifier. The tables become: (rdf.tablePrefix)spo,(rdf.tablePrefix)po,(rdf.tablePrefix)osp
 - ac.* : Accumulo connection parameters
 - rdf.format : See RDFFormat from openrdf, samples include (Trig, N-Triples, RDF/XML)
-- io.sort.mb : Higher the value, the faster the job goes. Just remember that you will need this much ram at least per mapper
+- sc.use_freetext, sc.use_geo, sc.use_temporal, sc.use_entity : If any of these are set to true, statements will also be
+    added to the enabled secondary indices.
+- sc.freetext.predicates, sc.geo.predicates, sc.temporal.predicates: If the associated indexer is enabled, these options specify
+    which statements should be sent to that indexer (based on the predicate). If not given, all indexers will attempt to index
+    all statements.
 
-The argument is the directory/file to load. This file needs to be loaded into HDFS before running.
+The argument is the directory/file to load. This file needs to be loaded into HDFS before running. If loading a directory, all files should have the same RDF
+format.
 
 ## Direct OpenRDF API
 
@@ -139,4 +144,4 @@ conn.commit();
 
 conn.close();
 myRepository.shutDown();
-```
\ No newline at end of file
+```

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/markdown/mapreduce.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/mapreduce.md b/extras/rya.manual/src/site/markdown/mapreduce.md
new file mode 100644
index 0000000..fde2231
--- /dev/null
+++ b/extras/rya.manual/src/site/markdown/mapreduce.md
@@ -0,0 +1,107 @@
+<!--
+
+[comment]: # Licensed to the Apache Software Foundation (ASF) under one
+[comment]: # or more contributor license agreements.  See the NOTICE file
+[comment]: # distributed with this work for additional information
+[comment]: # regarding copyright ownership.  The ASF licenses this file
+[comment]: # to you under the Apache License, Version 2.0 (the
+[comment]: # "License"); you may not use this file except in compliance
+[comment]: # with the License.  You may obtain a copy of the License at
+[comment]: #
+[comment]: #   http://www.apache.org/licenses/LICENSE-2.0
+[comment]: #
+[comment]: # Unless required by applicable law or agreed to in writing,
+[comment]: # software distributed under the License is distributed on an
+[comment]: # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+[comment]: # KIND, either express or implied.  See the License for the
+[comment]: # specific language governing permissions and limitations
+[comment]: # under the License.
+
+-->
+# MapReduce Interface
+
+The rya.mapreduce project contains a set of classes facilitating the use of
+Accumulo-backed Rya as the input source or output destination of Hadoop
+MapReduce jobs.
+
+## Writable
+
+*RyaStatementWritable* wraps a statement in a WritableComparable object, so
+triples can be used as keys or values in MapReduce tasks. Statements are
+considered equal if they contain equivalent triples and equivalent Accumulo
+metadata (visibility, timestamp, etc.).
+
+## Statement Input
+
+Input formats are provided for reading triple data from Rya or from RDF files:
+
+- *RdfFileInputFormat* will read and parse RDF files of any format. Format must
+  be explicitly specified. Reading and parsing is done asynchronously, enabling
+  large input files depending on how much information the openrdf parser itself
+  needs to hold in memory in order to parse the file. (For example, large
+  N-Triples files can be handled easily, but large XML files might require you
+  to allocate more memory for the Map task.) Handles multiple files if given a
+  directory as input, as long as all files are the specified format. Files will
+  only be split if the format is set to N-Triples or N-Quads; otherwise, the
+  number of input files will be the number of splits. Output pairs are
+  `<LongWritable, RyaStatementWritable>`, where the former is the number of the
+  statement within the input split and the latter is the statement itself.
+
+- *RyaInputFormat* will read statements directly from a Rya table in Accumulo.
+  Extends Accumulo's AbstractInputFormat and uses that class's configuration
+  methods to configure the connection to Accumulo. The table scanned should be
+  one of the Rya core tables (spo, po, or osp), and whichever is used should be
+  specified using `RyaInputFormat.setTableLayout`, so the input format can
+  deserialize the statements correctly. Choice of table may influence
+  parallelization if the tables are split differently in Accumulo. (The number
+  of splits in Accumulo will be the number of input splits in Hadoop and
+  therefore the number of Mappers.) Output pairs are
+  `<Text, RyaStatementWritable>`, where the former is the Accumulo row ID and
+  the latter is the statement itself.
+
+## Statement Output
+
+An output format is provided for writing triple data to Rya:
+
+- *RyaOutputFormat* will insert statements into the Rya core tables and/or any
+  configured secondary indexers. Configuration options include:
+    * Table prefix: identifies Rya instance
+    * Default visibility: any statement without a visibility set will be written
+      with this visibility
+    * Default context: any statement without a context (named graph) set will be
+      written with this context
+    * Enable freetext index, geo index, temporal index, entity index, and core
+      tables: separate options for configuring exactly which indexers to use.
+      If using secondary indexers, consider providing configuration variables
+      "sc.freetext.predicates", "sc.geo.predicates", and "sc.temporal.predicates"
+      as appropriate; otherwise each indexer will attempt to index every
+      statement.
+  Expects input pairs `<Writable, RyaStatementWritable>`. Keys are ignored and
+  values are written to Rya.
+
+## Configuration
+
+*MRUtils* defines constant configuration parameter names used for passing
+Accumulo connection information, Rya prefix and table layout, RDF format,
+etc., as well as some convenience methods for getting and setting these
+values with respect to a given Configuration.
+
+## Base Tool
+
+*AbstractAccumuloMRTool* can be used as a base class for Rya MapReduce Tools
+using the ToolRunner API. It extracts necessary parameters from the
+configuration and provides methods for setting input and/or output formats and
+configuring them accordingly. To use, extend this class and implement `run`.
+In the run method, call `init` to extract and validate configuration values from
+the Hadoop Configuration. Then use `setup*(Input/Output)` methods as needed to
+configure input and output for MapReduce jobs using the stored parameters.
+(Input and output formats can then be configured directly, if necessary.)
+
+Expects parameters to be specified in the configuration using the names defined
+in MRUtils, or for secondary indexers, the names in
+`mvm.rya.indexing.accumulo.ConfigUtils`.
+
+## Examples
+
+See the `examples` subpackage for examples of how to use the interface, and the
+`tools` subpackage for some individual MapReduce applications.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/markdown/sm-firststeps.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/sm-firststeps.md b/extras/rya.manual/src/site/markdown/sm-firststeps.md
index 34f995b..431292a 100644
--- a/extras/rya.manual/src/site/markdown/sm-firststeps.md
+++ b/extras/rya.manual/src/site/markdown/sm-firststeps.md
@@ -42,11 +42,11 @@ See the [Build From Source Section](build-source.md) to get the appropriate arti
 I find that the best way to load the data is through the Bulk Load Map Reduce job.
 
 * Save the RDF Data above onto HDFS. From now on we will refer to this location as `<RDF_HDFS_LOCATION>`
-* Move the `accumulo.rya-<version>-job.jar` onto the hadoop cluster
+* Move the `rya.mapreduce-<version>-job.jar` onto the hadoop cluster
 * Bulk load the data. Here is a sample command line:
 
 ```
-hadoop jar ../accumulo.rya-2.0.0-SNAPSHOT-job.jar BulkNtripsInputTool -Drdf.tablePrefix=lubm_ -Dcb.username=user -Dcb.pwd=cbpwd -Dcb.instance=instance -Dcb.zk=zookeeperLocation -Drdf.format=N-Triples <RDF_HDFS_LOCATION>
+hadoop jar ../rya.mapreduce-3.2.10-SNAPSHOT-job.jar mvm.rya.accumulo.mr.RdfFileInputTool -Drdf.tablePrefix=lubm_ -Dcb.username=user -Dcb.pwd=cbpwd -Dcb.instance=instance -Dcb.zk=zookeeperLocation -Drdf.format=N-Triples <RDF_HDFS_LOCATION>
 ```
 
 Once the data is loaded, it is actually a good practice to compact your tables. You can do this by opening the accumulo shell `shell` and running the `compact` command on the generated tables. Remember the generated tables will be prefixed by the `rdf.tablePrefix` property you assigned above. The default tablePrefix is `rts`.
@@ -77,4 +77,4 @@ This page provides a very simple text box for running queries against the store
 
 Remember to update the connection information in the WAR: `WEB-INF/spring/spring-accumulo.xml`
 
-See the [Query data section](querydata.md) for more information.
\ No newline at end of file
+See the [Query data section](querydata.md) for more information.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.manual/src/site/site.xml
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/site.xml b/extras/rya.manual/src/site/site.xml
index a671d3d..a5fab57 100644
--- a/extras/rya.manual/src/site/site.xml
+++ b/extras/rya.manual/src/site/site.xml
@@ -45,6 +45,7 @@ under the License.
         <item name="Evaluation Table" href="eval.html"/>
         <item name="Pre-computed Joins" href="loadPrecomputedJoin.html"/>
         <item name="Inferencing" href="infer.html"/>
+        <item name="MapReduce Interface" href="mapreduce.html"/>
     </menu>
 
     <menu name="Samples">

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.reasoning/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.reasoning/pom.xml b/extras/rya.reasoning/pom.xml
index b7b7293..bc00404 100644
--- a/extras/rya.reasoning/pom.xml
+++ b/extras/rya.reasoning/pom.xml
@@ -43,6 +43,10 @@ under the License.
             <groupId>org.apache.rya</groupId>
             <artifactId>rya.sail</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.mapreduce</artifactId>
+        </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/AbstractReasoningTool.java
----------------------------------------------------------------------
diff --git a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/AbstractReasoningTool.java b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/AbstractReasoningTool.java
index dde83c6..09b4a16 100644
--- a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/AbstractReasoningTool.java
+++ b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/AbstractReasoningTool.java
@@ -22,8 +22,8 @@ package mvm.rya.reasoning.mr;
 import java.io.IOException;
 
 import mvm.rya.accumulo.mr.RyaStatementWritable;
-import mvm.rya.accumulo.mr.fileinput.RdfFileInputFormat;
-import mvm.rya.accumulo.mr.utils.MRUtils;
+import mvm.rya.accumulo.mr.RdfFileInputFormat;
+import mvm.rya.accumulo.mr.MRUtils;
 import mvm.rya.reasoning.Derivation;
 import mvm.rya.reasoning.Fact;
 import mvm.rya.reasoning.Schema;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/ConformanceTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/ConformanceTest.java b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/ConformanceTest.java
index 02cce66..0209eff 100644
--- a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/ConformanceTest.java
+++ b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/ConformanceTest.java
@@ -33,7 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import mvm.rya.accumulo.mr.utils.MRUtils;
+import mvm.rya.accumulo.mr.MRUtils;
 import mvm.rya.reasoning.Fact;
 import mvm.rya.reasoning.Schema;
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/MRReasoningUtils.java
----------------------------------------------------------------------
diff --git a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/MRReasoningUtils.java b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/MRReasoningUtils.java
index 3bed4ca..b306ee8 100644
--- a/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/MRReasoningUtils.java
+++ b/extras/rya.reasoning/src/main/java/mvm/rya/reasoning/mr/MRReasoningUtils.java
@@ -25,7 +25,7 @@ import java.io.IOException;
 import mvm.rya.accumulo.AccumuloRdfConfiguration;
 import mvm.rya.accumulo.AccumuloRdfConstants;
 import mvm.rya.accumulo.AccumuloRyaDAO;
-import mvm.rya.accumulo.mr.utils.MRUtils;
+import mvm.rya.accumulo.mr.MRUtils;
 import mvm.rya.api.RdfCloudTripleStoreConstants;
 import mvm.rya.api.RdfCloudTripleStoreUtils;
 import mvm.rya.api.domain.RyaStatement;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/tinkerpop.rya/pom.xml
----------------------------------------------------------------------
diff --git a/extras/tinkerpop.rya/pom.xml b/extras/tinkerpop.rya/pom.xml
index 9ccb4b1..b92eb96 100644
--- a/extras/tinkerpop.rya/pom.xml
+++ b/extras/tinkerpop.rya/pom.xml
@@ -39,6 +39,10 @@ under the License.
             <groupId>org.apache.rya</groupId>
             <artifactId>accumulo.rya</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.mapreduce</artifactId>
+        </dependency>
         
         <dependency>
             <groupId>com.tinkerpop.gremlin</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/tinkerpop.rya/src/main/groovy/mvm/rya/blueprints/config/RyaGraphConfiguration.groovy
----------------------------------------------------------------------
diff --git a/extras/tinkerpop.rya/src/main/groovy/mvm/rya/blueprints/config/RyaGraphConfiguration.groovy b/extras/tinkerpop.rya/src/main/groovy/mvm/rya/blueprints/config/RyaGraphConfiguration.groovy
index fc3419d..a6e906c 100644
--- a/extras/tinkerpop.rya/src/main/groovy/mvm/rya/blueprints/config/RyaGraphConfiguration.groovy
+++ b/extras/tinkerpop.rya/src/main/groovy/mvm/rya/blueprints/config/RyaGraphConfiguration.groovy
@@ -29,7 +29,7 @@ import mvm.rya.blueprints.sail.RyaSailGraph
 import mvm.rya.rdftriplestore.RdfCloudTripleStore
 import mvm.rya.rdftriplestore.inference.InferenceEngine
 import org.apache.commons.configuration.Configuration
-import static mvm.rya.accumulo.mr.utils.MRUtils.*
+import static mvm.rya.accumulo.mr.MRUtils.*
 import org.apache.commons.configuration.MapConfiguration
 import mvm.rya.blueprints.sail.RyaSailEdge
 import mvm.rya.blueprints.sail.RyaSailVertex

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/TstGremlinRya.groovy
----------------------------------------------------------------------
diff --git a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/TstGremlinRya.groovy b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/TstGremlinRya.groovy
index fe0f4e0..ec8beb8 100644
--- a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/TstGremlinRya.groovy
+++ b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/TstGremlinRya.groovy
@@ -30,7 +30,7 @@
 //import mvm.rya.rdftriplestore.RdfCloudTripleStore
 //import mvm.rya.rdftriplestore.inference.InferenceEngine
 //import org.apache.accumulo.core.client.ZooKeeperInstance
-//import static mvm.rya.accumulo.mr.utils.MRUtils.*
+//import static mvm.rya.accumulo.mr.MRUtils.*
 //import static mvm.rya.api.RdfCloudTripleStoreConfiguration.CONF_QUERYPLAN_FLAG
 //import static mvm.rya.api.RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX
 //

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/config/RyaGraphConfigurationTest.groovy
----------------------------------------------------------------------
diff --git a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/config/RyaGraphConfigurationTest.groovy b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/config/RyaGraphConfigurationTest.groovy
index 9dd0627..c4f5dbb 100644
--- a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/config/RyaGraphConfigurationTest.groovy
+++ b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/config/RyaGraphConfigurationTest.groovy
@@ -30,7 +30,7 @@ import org.openrdf.model.impl.StatementImpl
 import org.openrdf.model.impl.ValueFactoryImpl
 
 import static mvm.rya.api.RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX
-import static mvm.rya.accumulo.mr.utils.MRUtils.*
+import static mvm.rya.accumulo.mr.MRUtils.*
 import org.apache.accumulo.core.security.Authorizations
 import org.apache.accumulo.core.client.Connector
 import mvm.rya.accumulo.AccumuloRyaDAO

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/sail/RyaSailVertexSequenceTest.groovy
----------------------------------------------------------------------
diff --git a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/sail/RyaSailVertexSequenceTest.groovy b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/sail/RyaSailVertexSequenceTest.groovy
index c661350..f0d2481 100644
--- a/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/sail/RyaSailVertexSequenceTest.groovy
+++ b/extras/tinkerpop.rya/src/test/groovy/mvm/rya/blueprints/sail/RyaSailVertexSequenceTest.groovy
@@ -23,7 +23,7 @@ import mvm.rya.api.utils.IteratorWrapper
 import junit.framework.TestCase
 import mvm.rya.blueprints.config.RyaGraphConfiguration
 import org.openrdf.model.Statement
-import static mvm.rya.accumulo.mr.utils.MRUtils.*
+import static mvm.rya.accumulo.mr.MRUtils.*
 import static mvm.rya.api.RdfCloudTripleStoreConstants.VALUE_FACTORY
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/42895eac/mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/mapreduce/pom.xml b/mapreduce/pom.xml
new file mode 100644
index 0000000..40dd1df
--- /dev/null
+++ b/mapreduce/pom.xml
@@ -0,0 +1,125 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya-project</artifactId>
+        <version>3.2.10-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>rya.mapreduce</artifactId>
+    <name>Apache Rya MapReduce Tools</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>accumulo.rya</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.indexing</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.accumulo</groupId>
+            <artifactId>accumulo-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-ntriples</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-nquads</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-trig</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <version>1.1.0</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.rat</groupId>
+                    <artifactId>apache-rat-plugin</artifactId>
+                    <configuration>
+                        <excludes>
+                            <!-- RDF data Files -->
+                            <exclude>**/*.ntriples</exclude>
+                            <exclude>**/*.trig</exclude>
+                        </excludes>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>mr</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-shade-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <configuration>
+                                    <transformers>
+                                        <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                    </transformers>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>