You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by ca...@apache.org on 2017/04/14 13:47:17 UTC

[3/3] incubator-rya git commit: RYA-226-ConfigBuilders, de-duplication of some config keys, and config documentation. Closes #127

RYA-226-ConfigBuilders, de-duplication of some config keys, and config documentation. Closes #127


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

Branch: refs/heads/master
Commit: be9ea9a37690ec55e8f1057136903a406541c64b
Parents: bd9b912
Author: Caleb Meier <ca...@parsons.com>
Authored: Thu Dec 8 08:56:03 2016 -0800
Committer: Caleb Meier <ca...@parsons.com>
Committed: Fri Apr 14 06:45:34 2017 -0700

----------------------------------------------------------------------
 .../api/RdfCloudTripleStoreConfiguration.java   |  54 +-
 ...RdfCloudTripleStoreConfigurationBuilder.java | 156 +++++
 .../rya/api/RdfCloudTripleStoreConstants.java   |   3 +-
 .../rya/api/instance/ConfigurationFields.java   |   4 +-
 .../api/instance/RyaDetailsToConfiguration.java |  10 +-
 .../instance/RyaDetailsToConfigurationTest.java |   4 +-
 ...AbstractAccumuloRdfConfigurationBuilder.java | 208 ++++++
 .../rya/accumulo/AccumuloRdfConfiguration.java  | 157 ++++-
 .../AccumuloRdfConfigurationBuilder.java        | 105 +++
 .../accumulo/AccumuloRdfConfigurationTest.java  |  89 ++-
 .../test/resources/properties/rya.properties    |  30 +
 .../AbstractMongoDBRdfConfigurationBuilder.java | 177 +++++
 .../rya/mongodb/MongoDBRdfConfiguration.java    | 162 ++++-
 .../mongodb/MongoDBRdfConfigurationBuilder.java | 107 +++
 .../mongodb/MongoDBRdfConfigurationTest.java    | 103 +++
 .../src/test/resources/rya.properties           |  29 +
 .../api/client/accumulo/AccumuloInstall.java    |   1 +
 .../accumulo/AccumuloIndexingConfiguration.java | 658 +++++++++++++++++++
 .../rya/indexing/accumulo/ConfigUtils.java      | 168 +++--
 .../external/PrecomputedJoinIndexerConfig.java  |   3 +-
 .../external/fluo/FluoPcjUpdaterConfig.java     |   4 +-
 .../mongodb/MongoIndexingConfiguration.java     | 418 ++++++++++++
 .../rya/api/client/accumulo/FluoITBase.java     |   1 -
 .../AccumuloIndexingConfigurationTest.java      | 154 +++++
 .../mongo/MongoIndexingConfigurationTest.java   | 119 ++++
 .../resources/accumulo_rya_indexing.properties  |  40 ++
 .../resources/mongo_rya_indexing.properties     |  33 +
 .../src/main/java/EntityDirectExample.java      |  22 +-
 .../src/main/java/MongoRyaDirectExample.java    |  34 +-
 .../src/main/java/RyaClientExample.java         |  53 +-
 .../src/main/java/RyaDirectExample.java         |  58 +-
 .../apache/rya/accumulo/mr/merge/MergeTool.java |   4 +-
 .../rya/indexing/pcj/fluo/api/CreatePcj.java    |   3 +-
 .../apache/rya/indexing/pcj/fluo/ITBase.java    |   5 +-
 .../pcj/fluo/integration/StreamingTestIT.java   |   5 +-
 .../org/apache/rya/camel/cbsail/CbSailTest.java |   2 +-
 36 files changed, 2925 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
index ae2e03a..c72b85a 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
@@ -72,59 +72,6 @@ public abstract class RdfCloudTripleStoreConfiguration extends Configuration {
     public static final String CONF_PCJ_TABLES = "pcj.index.tables";
     public static final String CONF_STATEMENT_METADATA_PROPERTIES = "statement.metadata.properites";
     public static final String CONF_USE_STATEMENT_METADATA = "use.statement.metadata";
-
-
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_DISP_QUERYPLAN = CONF_QUERYPLAN_FLAG;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_AUTH = CONF_QUERY_AUTH;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_CV = CONF_CV;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_TTL = CONF_TTL;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_STARTTIME = CONF_STARTTIME;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_PERFORMANT = CONF_PERFORMANT;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_INFER = CONF_INFER;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_USESTATS = CONF_USE_STATS;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_OFFSET = CONF_OFFSET;
-    /**
-     * @deprecated use CONF_*
-     */
-    @Deprecated
-	public static final String BINDING_LIMIT = CONF_LIMIT;
-
     public static final String STATS_PUSH_EMPTY_RDFTYPE_DOWN = "conf.stats.rdftype.down";
     public static final String INFER_INCLUDE_INVERSEOF = "infer.include.inverseof";
     public static final String INFER_INCLUDE_SUBCLASSOF = "infer.include.subclassof";
@@ -341,6 +288,7 @@ public abstract class RdfCloudTripleStoreConfiguration extends Configuration {
     }
 
     public void setAuths(String... auths) {
+        Preconditions.checkNotNull(auths);
         setStrings(CONF_QUERY_AUTH, auths);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfigurationBuilder.java b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfigurationBuilder.java
new file mode 100644
index 0000000..a39464c
--- /dev/null
+++ b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfigurationBuilder.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This is a base class meant to be extended by Rya configuration builders. Any
+ * class extending this class inherits setter methods that should be common to
+ * all Rya implementations.
+ * 
+ * @param <B>
+ *            the configuration builder returned by each of the setter methods
+ * @param <C>
+ *            the configuration object returned by the builder when build is
+ *            called
+ */
+public abstract class RdfCloudTripleStoreConfigurationBuilder<B extends RdfCloudTripleStoreConfigurationBuilder<B, C>, C extends RdfCloudTripleStoreConfiguration> {
+
+    private String prefix = "rya_";
+    private String auths;
+    private String visibilities;
+    private boolean useInference = false;
+    private boolean displayPlan = false;
+
+    /**
+     * Auxiliary method to return specified Builder in a type safe way
+     * 
+     * @return specified builder
+     */
+    protected abstract B confBuilder();
+
+    /**
+     * Auxiliary method to return type of configuration object constructed by
+     * builder in a type safe way
+     * 
+     * @return specified configuration
+     */
+    protected abstract C createConf();
+
+    /**
+     * Set whether to use backwards chaining inferencing during query
+     * evaluation. The default value is false.
+     * 
+     * @param useInference
+     *            - turn inferencing on and off in Rya
+     * @return B - concrete builder class for chaining method invocations
+     */
+    public B setUseInference(boolean useInference) {
+        this.useInference = useInference;
+        return confBuilder();
+    }
+
+    /**
+     * 
+     * Sets the authorization for querying the underlying data store.
+     * 
+     * @param auths
+     *            - authorizations for querying underlying datastore
+     * @return B - concrete builder class for chaining method invocations
+     */
+    public B setAuths(String auths) {
+        this.auths = auths;
+        return confBuilder();
+    }
+
+    /**
+     * Set the column visibities for ingested triples. If no value is set,
+     * triples won't have a visibility.
+     * 
+     * @param visibilities
+     *            - visibilities assigned to any triples inserted into Rya
+     * @return B - concrete builder class for chaining method invocations
+     */
+    public B setVisibilities(String visibilites) {
+        this.visibilities = visibilites;
+        return confBuilder();
+    }
+
+    /**
+     *
+     * Sets the prefix for the Rya instance to connect to. This parameter is
+     * required and the default value is "rya_"
+     * 
+     * @param prefix
+     *            - the prefix for the Rya instance
+     * @return B - concrete builder class for chaining method invocations
+     */
+    public B setRyaPrefix(String prefix) {
+        this.prefix = prefix;
+        return confBuilder();
+    }
+
+    /**
+     * Set whether to display query plan during optimization. The default value
+     * is false.
+     * 
+     * @param displayPlan
+     *            - display the parsed query plan during query evaluation
+     *            (useful for debugging)
+     * @return B - concrete builder class for chaining method invocations
+     */
+    public B setDisplayQueryPlan(boolean displayPlan) {
+        this.displayPlan = displayPlan;
+        return confBuilder();
+    }
+
+    /**
+     * 
+     * @return {@link RdfCloudTripleStoreConfiguration} object with specified parameters set
+     */
+    public C build() {
+        return getConf(createConf());
+    }
+
+    /**
+     * Assigns builder values to appropriate parameters within the {@link Configuration} object.
+     * 
+     * @param conf - Configuration object
+     * @return - Configuration object with parameters set
+     */
+    private C getConf(C conf) {
+
+        conf.setInfer(useInference);
+        conf.setTablePrefix(prefix);
+        conf.setInt("sc.cloudbase.numPartitions", 3);
+        conf.setAuths(auths);
+        if (visibilities != null) {
+            conf.setCv(visibilities);
+        }
+        conf.setDisplayQueryPlan(displayPlan);
+
+        return conf;
+    }
+
+    protected static boolean getBoolean(String boolString) {
+        return Boolean.parseBoolean(boolString);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConstants.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConstants.java b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConstants.java
index 5019a6e..2092951 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConstants.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConstants.java
@@ -1,5 +1,7 @@
 package org.apache.rya.api;
 
+import org.apache.hadoop.io.Text;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -24,7 +26,6 @@ package org.apache.rya.api;
 import org.apache.rya.api.domain.RyaSchema;
 import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaURI;
-import org.apache.hadoop.io.Text;
 import org.openrdf.model.Literal;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/main/java/org/apache/rya/api/instance/ConfigurationFields.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/instance/ConfigurationFields.java b/common/rya.api/src/main/java/org/apache/rya/api/instance/ConfigurationFields.java
index b59a04e..b236d30 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/instance/ConfigurationFields.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/instance/ConfigurationFields.java
@@ -31,5 +31,7 @@ class ConfigurationFields {
     static final String USE_FREETEXT = "sc.use_freetext";
     static final String USE_TEMPORAL = "sc.use_temporal";
     static final String USE_ENTITY = "sc.use_entity";
-    static final String USE_PCJ = "sc.use_pcj";
+    static final String USE_PCJ_UPDATER = "sc.use.updater";
+    static final String FLUO_APP_NAME = "rya.indexing.pcj.fluo.fluoAppName";
+    static final String PCJ_UPDATER_TYPE = "rya.indexing.pcj.updaterType";
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java b/common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java
index adc2498..b26fb10 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java
@@ -25,6 +25,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
+import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails;
 
 import com.google.common.base.Optional;
 
@@ -52,7 +53,14 @@ public class RyaDetailsToConfiguration {
         checkAndSet(conf, ConfigurationFields.USE_FREETEXT, details.getFreeTextIndexDetails().isEnabled());
       //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
         checkAndSet(conf, ConfigurationFields.USE_TEMPORAL, details.getTemporalIndexDetails().isEnabled());
-        checkAndSet(conf, ConfigurationFields.USE_PCJ, details.getPCJIndexDetails().isEnabled());
+        PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
+		if (pcjDetails.isEnabled() && pcjDetails.getFluoDetails().isPresent()) {
+			checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, true);
+			conf.set(ConfigurationFields.FLUO_APP_NAME, pcjDetails.getFluoDetails().get().getUpdateAppName());
+			conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "FLUO");
+		} else {
+			checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, false);
+		}
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/common/rya.api/src/test/java/org/apache/rya/api/instance/RyaDetailsToConfigurationTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/instance/RyaDetailsToConfigurationTest.java b/common/rya.api/src/test/java/org/apache/rya/api/instance/RyaDetailsToConfigurationTest.java
index 0cb5c77..2103e1f 100644
--- a/common/rya.api/src/test/java/org/apache/rya/api/instance/RyaDetailsToConfigurationTest.java
+++ b/common/rya.api/src/test/java/org/apache/rya/api/instance/RyaDetailsToConfigurationTest.java
@@ -21,7 +21,7 @@ package org.apache.rya.api.instance;
 
 import static org.apache.rya.api.instance.ConfigurationFields.USE_ENTITY;
 import static org.apache.rya.api.instance.ConfigurationFields.USE_FREETEXT;
-import static org.apache.rya.api.instance.ConfigurationFields.USE_PCJ;
+import static org.apache.rya.api.instance.ConfigurationFields.USE_PCJ_UPDATER;
 import static org.apache.rya.api.instance.ConfigurationFields.USE_TEMPORAL;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -76,6 +76,6 @@ public class RyaDetailsToConfigurationTest {
         assertFalse(conf.getBoolean(USE_FREETEXT, true));
       //RYA-215assertTrue(conf.getBoolean(USE_GEO, false));
         assertTrue(conf.getBoolean(USE_TEMPORAL, false));
-        assertTrue(conf.getBoolean(USE_PCJ, false));
+        assertTrue(conf.getBoolean(USE_PCJ_UPDATER, false));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AbstractAccumuloRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AbstractAccumuloRdfConfigurationBuilder.java b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AbstractAccumuloRdfConfigurationBuilder.java
new file mode 100644
index 0000000..e342db2
--- /dev/null
+++ b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AbstractAccumuloRdfConfigurationBuilder.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.accumulo;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.api.RdfCloudTripleStoreConfigurationBuilder;
+
+/**
+ * This builder class will set all of the core Accumulo-backed Rya configuration
+ * parameters. Any builder extending this class will have setter methods for all
+ * of the necessary parameters to connect to an Accumulo backed Rya instance.
+ *
+ * @param <B>
+ *            - builder returned by setter methods extending this class
+ * @param <C>
+ *            - configuration object constructed by the builder extending this
+ *            class
+ */
+public abstract class AbstractAccumuloRdfConfigurationBuilder<B extends AbstractAccumuloRdfConfigurationBuilder<B, C>, C extends AccumuloRdfConfiguration>
+        extends RdfCloudTripleStoreConfigurationBuilder<B, C> {
+
+    private String user;
+    private String pass;
+    private String zoo;
+    private String instance;
+    private boolean useMock = false;
+    private boolean usePrefixHashing = false;
+    private boolean useComposite = false;
+    private boolean useSelectivity = false;
+
+    protected static final String ACCUMULO_USER = "accumulo.user";
+    protected static final String ACCUMULO_PASSWORD = "accumulo.password";
+    protected static final String ACCUMULO_INSTANCE = "accumulo.instance";
+    protected static final String ACCUMULO_AUTHS = "accumulo.auths";
+    protected static final String ACCUMULO_VISIBILITIES = "accumulo.visibilities";
+    protected static final String ACCUMULO_ZOOKEEPERS = "accumulo.zookeepers";
+    protected static final String ACCUMULO_RYA_PREFIX = "accumulo.rya.prefix";
+    protected static final String USE_INFERENCE = "use.inference";
+    protected static final String USE_DISPLAY_QUERY_PLAN = "use.display.plan";
+    protected static final String USE_MOCK_ACCUMULO = "use.mock";
+    protected static final String USE_PREFIX_HASHING = "use.prefix.hashing";
+    protected static final String USE_COUNT_STATS = "use.count.stats";
+    protected static final String USE_JOIN_SELECTIVITY = "use.join.selectivity";
+
+    /**
+     * Sets Accumulo user. This is a required parameter to connect to an
+     * Accumulo Rya Instance
+     * 
+     * @param user
+     *            - Accumulo user
+     */
+    public B setAccumuloUser(String user) {
+        this.user = user;
+        return confBuilder();
+    }
+
+    /**
+     * Sets Accumulo user's password. This is a required parameter to connect to
+     * an Accumulo Rya Instance
+     * 
+     * @param password
+     *            - password for Accumulo user specified by
+     *            {@link AbstractAccumuloRdfConfigurationBuilder#setAccumuloUser(String)}
+     * @return specified builder for chaining method invocations
+     */
+    public B setAccumuloPassword(String password) {
+        this.pass = password;
+        return confBuilder();
+    }
+
+    /**
+     * Sets name of Accumulo instance containing Rya tables. This is a required
+     * parameter to connect to an Accumulo Rya instance.
+     * 
+     * @param instance
+     *            - Accumulo instance name
+     * @return specified builder for chaining method invocations
+     */
+    public B setAccumuloInstance(String instance) {
+        this.instance = instance;
+        return confBuilder();
+    }
+
+    /**
+     * Sets Accumulo Zookeepers for instance specified by
+     * {@link AbstractAccumuloRdfConfigurationBuilder#setAccumuloInstance(String)}
+     * . This is a required parameter if
+     * {@link AbstractAccumuloRdfConfigurationBuilder#useMock} is false.
+     * 
+     * @param zoo
+     *            - Accumuo Zookeepers
+     * @return specified builder for chaining method invocations
+     */
+    public B setAccumuloZooKeepers(String zoo) {
+        this.zoo = zoo;
+        return confBuilder();
+    }
+
+    /**
+     * Specifies whether or not to use a mock version of Accumulo for the Rya
+     * Instance. The default value is false. If
+     * {@link AbstractAccumuloRdfConfigurationBuilder#useMock} is false, then
+     * Accumulo Zookeepers must be specified.
+     * 
+     * @param useMock
+     * @return specified builder for chaining method invocations
+     */
+    public B setUseMockAccumulo(boolean useMock) {
+        this.useMock = useMock;
+        return confBuilder();
+    }
+
+    /**
+     * Specifies whether to use prefix hashing as a table design optimization to
+     * prevent hot spotting. The default value is false.
+     * 
+     * @param useHash
+     *            - use prefix hashing to prevent hot spotting
+     * @return specified builder for chaining method invocations
+     */
+    public B setUseAccumuloPrefixHashing(boolean useHash) {
+        this.usePrefixHashing = useHash;
+        return confBuilder();
+    }
+
+    /**
+     * Specifies whether or not to use triple cardinalities for query
+     * optimization. This feature can only be used if the prospects table has
+     * been created by the prospector and the default value is false.
+     * 
+     * @param useCompositeCardinality
+     *            - use prospects statistics table for query optimization
+     * @return specified builder for chaining method invocations
+     */
+    public B setUseCompositeCardinality(boolean useCompositeCardinality) {
+        this.useComposite = useCompositeCardinality;
+        return confBuilder();
+    }
+
+    /**
+     * Specifies whether or not to use the join selectivity for query
+     * optimization. This feature can only be used if the selectivity table has
+     * been created and the default value is false.
+     * 
+     * @param useJoinSelectivity
+     *            - use join selectivity statistics table for query optimization
+     * @return specified builder for chaining method invocations
+     */
+    public B setUseJoinSelectivity(boolean useJoinSelectivity) {
+        this.useSelectivity = useJoinSelectivity;
+        return confBuilder();
+    }
+
+    /**
+     * @return extension of {@link AccumuloRdfConfiguration} with specified parameters set
+     */
+    public C build() {
+        return getConf(super.build());
+    }
+
+    /**
+     * Assigns builder values to appropriate parameters within the {@link Configuration} object.
+     * 
+     * @param conf - Configuration object
+     * @return - Configuration object with parameters set
+     */
+    private C getConf(C conf) {
+
+        conf.setAccumuloInstance(instance);
+        conf.setAccumuloPassword(pass);
+        conf.setAccumuloUser(user);
+        
+        if (!useMock) {
+            conf.setAccumuloZookeepers(zoo);
+        }
+
+        conf.setUseMockAccumulo(useMock);
+        conf.setPrefixRowsWithHash(usePrefixHashing);
+        
+        if (useSelectivity) {
+            conf.setUseStats(true);
+            conf.setCompositeCardinality(true);
+            conf.setUseSelectivity(useSelectivity);
+        } else if (useComposite) {
+            conf.setUseStats(true);
+            conf.setCompositeCardinality(useComposite);
+        }
+
+        return conf;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfiguration.java b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfiguration.java
index 87694f9..d988b20 100644
--- a/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfiguration.java
+++ b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfiguration.java
@@ -1,5 +1,3 @@
-package org.apache.rya.accumulo;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -8,9 +6,9 @@ package org.apache.rya.accumulo;
  * 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
@@ -18,25 +16,25 @@ package org.apache.rya.accumulo;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.accumulo;
 
-
-
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-
-import org.apache.rya.accumulo.experimental.AccumuloIndexer;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.accumulo.experimental.AccumuloIndexer;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
+
 /**
  * Created by IntelliJ IDEA.
  * Date: 4/25/12
@@ -45,6 +43,12 @@ import com.google.common.collect.Lists;
  */
 public class AccumuloRdfConfiguration extends RdfCloudTripleStoreConfiguration {
 
+    public static final String CLOUDBASE_INSTANCE = "sc.cloudbase.instancename";
+    public static final String CLOUDBASE_ZOOKEEPERS = "sc.cloudbase.zookeepers";
+    public static final String CLOUDBASE_USER = "sc.cloudbase.username";
+    public static final String CLOUDBASE_PASSWORD = "sc.cloudbase.password";
+    public static final String USE_MOCK_INSTANCE = ".useMockInstance";
+    
     public static final String MAXRANGES_SCANNER = "ac.query.maxranges";
     
     public static final String CONF_ADDITIONAL_INDEXERS = "ac.additional.indexers";
@@ -68,10 +72,143 @@ public class AccumuloRdfConfiguration extends RdfCloudTripleStoreConfiguration {
         super(other);
     }
 
+    public AccumuloRdfConfigurationBuilder getBuilder() {
+    	return new AccumuloRdfConfigurationBuilder();
+    }
+    
+    /**
+     * Creates an AccumuloRdfConfiguration object from a Properties file.  This method assumes
+     * that all values in the Properties file are Strings and that the Properties file uses the keys below.
+     * See accumulo/rya/src/test/resources/properties/rya.properties for an example.
+     * <br>
+     * <ul>
+     * <li>"accumulo.auths" - String of Accumulo authorizations. Default is empty String.
+     * <li>"accumulo.visibilities" - String of Accumulo visibilities assigned to ingested triples.
+     * <li>"accumulo.instance" - Accumulo instance name (required)
+     * <li>"accumulo.user" - Accumulo user (required)
+     * <li>"accumulo.password" - Accumulo password (required)
+     * <li>"accumulo.rya.prefix" - Prefix for Accumulo backed Rya instance.  Default is "rya_"
+     * <li>"accumulo.zookeepers" - Zookeepers for underlying Accumulo instance (required if not using Mock)
+     * <li>"use.mock" - Use a MockAccumulo instance as back-end for Rya instance.  Default is false.
+     * <li>"use.prefix.hashing" - Use prefix hashing for triples.  Helps avoid hot-spotting.  Default is false.
+     * <li>"use.count.stats" - Use triple pattern cardinalities for query optimization.   Default is false.
+     * <li>"use.join.selectivity" - Use join selectivity for query optimization.  Default is false.
+     * <li>"use.display.plan" - Display query plan during evaluation.  Useful for debugging.   Default is true.
+     * <li>"use.inference" - Use backward chaining inference during query evaluation.   Default is false.
+     * </ul>
+     * <br>
+     * @param props - Properties file containing Accumulo specific configuration parameters
+     * @return AccumumuloRdfConfiguration with properties set
+     */
+    
+    public static AccumuloRdfConfiguration fromProperties(Properties props) {
+    	return AccumuloRdfConfigurationBuilder.fromProperties(props).build();
+    }
+    
     @Override
     public AccumuloRdfConfiguration clone() {
         return new AccumuloRdfConfiguration(this);
     }
+    
+    /**
+     * Sets the Accumulo username from the configuration object that is meant to
+     * be used when connecting a {@link Connector} to Accumulo.
+     *
+     */
+    public void setAccumuloUser(String user) {
+    	Preconditions.checkNotNull(user);
+    	set(CLOUDBASE_USER, user);
+    }
+    
+    /**
+     * Get the Accumulo username from the configuration object that is meant to
+     * be used when connecting a {@link Connector} to Accumulo.
+     *
+     * @return The username if one could be found; otherwise {@code null}.
+     */
+    public String getAccumuloUser(){
+    	return get(CLOUDBASE_USER); 
+    }
+    
+    /**
+     * Sets the Accumulo password from the configuration object that is meant to
+     * be used when connecting a {@link Connector} to Accumulo.
+     *
+     */
+    public void setAccumuloPassword(String password) {
+    	Preconditions.checkNotNull(password);
+    	set(CLOUDBASE_PASSWORD, password);
+    }
+    
+    /**
+     * Get the Accumulo password from the configuration object that is meant to
+     * be used when connecting a {@link Connector} to Accumulo.
+     *
+     * @return The password if one could be found; otherwise an empty string.
+     */
+    public String getAccumuloPassword() {
+    	return get(CLOUDBASE_PASSWORD);
+    }
+    
+    /**
+     * Sets a comma delimited list of the names of the Zookeeper servers from
+     * the configuration object that is meant to be used when connecting a
+     * {@link Connector} to Accumulo.
+     *
+     */
+    public void setAccumuloZookeepers(String zookeepers) {
+    	Preconditions.checkNotNull(zookeepers);
+    	set(CLOUDBASE_ZOOKEEPERS, zookeepers);
+    }
+    
+    /**
+     * Get a comma delimited list of the names of the Zookeeper servers from
+     * the configuration object that is meant to be used when connecting a
+     * {@link Connector} to Accumulo.
+     *
+     * @return The zookeepers list if one could be found; otherwise {@code null}.
+     */
+    public String getAccumuloZookeepers() {
+    	return get(CLOUDBASE_ZOOKEEPERS);
+    }
+    
+    /**
+     * Sets the Accumulo instance name from the configuration object that is
+     * meant to be used when connecting a {@link Connector} to Accumulo.
+     *
+     */
+    public void setAccumuloInstance(String instance) {
+    	Preconditions.checkNotNull(instance);
+    	set(CLOUDBASE_INSTANCE, instance);
+    }
+    
+    /**
+     * Get the Accumulo instance name from the configuration object that is
+     * meant to be used when connecting a {@link Connector} to Accumulo.
+     *
+     * @return The instance name if one could be found; otherwise {@code null}.
+     */
+    public String getAccumuloInstance() {
+    	return get(CLOUDBASE_INSTANCE);
+    }
+    
+    /**
+     * Tells the Rya instance to use a Mock instance of Accumulo as its backing.
+     *
+     */
+    public void setUseMockAccumulo(boolean useMock) {
+    	setBoolean(USE_MOCK_INSTANCE, useMock);
+    }
+    
+    /**
+     * Indicates that a Mock instance of Accumulo is being used to back the Rya instance.
+     *
+     * @return {@code true} if the Rya instance is backed by a mock Accumulo; otherwise {@code false}.
+     */
+    public boolean getUseMockAccumulo() {
+    	return getBoolean(USE_MOCK_INSTANCE, false);
+    }
+    
 
     public Authorizations getAuthorizations() {
         String[] auths = getAuths();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfigurationBuilder.java b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfigurationBuilder.java
new file mode 100644
index 0000000..0640f65
--- /dev/null
+++ b/dao/accumulo.rya/src/main/java/org/apache/rya/accumulo/AccumuloRdfConfigurationBuilder.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */package org.apache.rya.accumulo;
+
+import java.util.Properties;
+
+/**
+ * This is a concrete extension of the
+ * {@link AbstractAccumuloRdfConfigurationBuilder} class which builds an
+ * {@link AccumuloRdfConfiguration} object. This builder creates an
+ * AccumuloRdfConfiguratio object and sets all of the parameters required to
+ * connect to an Accumulo Rya instance.
+ *
+ */
+public class AccumuloRdfConfigurationBuilder
+        extends AbstractAccumuloRdfConfigurationBuilder<AccumuloRdfConfigurationBuilder, AccumuloRdfConfiguration> {
+
+    /**
+     * Creates an AccumuloRdfConfiguration object from a Properties file. This
+     * method assumes that all values in the Properties file are Strings and
+     * that the Properties file uses the keys below. See
+     * accumulo/rya/src/test/resources/properties/rya.properties for an example.
+     * <br>
+     * <ul>
+     * <li>"accumulo.auths" - String of Accumulo authorizations. Default is
+     * empty String.
+     * <li>"accumulo.visibilities" - String of Accumulo visibilities assigned to
+     * ingested triples.
+     * <li>"accumulo.instance" - Accumulo instance name (required)
+     * <li>"accumulo.user" - Accumulo user (required)
+     * <li>"accumulo.password" - Accumulo password (required)
+     * <li>"accumulo.rya.prefix" - Prefix for Accumulo backed Rya instance.
+     * Default is "rya_"
+     * <li>"accumulo.zookeepers" - Zookeepers for underlying Accumulo instance
+     * (required if not using Mock)
+     * <li>"use.mock" - Use a MockAccumulo instance as back-end for Rya
+     * instance. Default is false.
+     * <li>"use.prefix.hashing" - Use prefix hashing for triples. Helps avoid
+     * hot-spotting. Default is false.
+     * <li>"use.count.stats" - Use triple pattern cardinalities for query
+     * optimization. Default is false.
+     * <li>"use.join.selectivity" - Use join selectivity for query optimization.
+     * Default is false.
+     * <li>"use.display.plan" - Display query plan during evaluation. Useful for
+     * debugging. Default is true.
+     * <li>"use.inference" - Use backward chaining inference during query
+     * evaluation. Default is false.
+     * </ul>
+     * <br>
+     * 
+     * @param props
+     *            - Properties file containing Accumulo specific configuration
+     *            parameters
+     * @return AccumumuloRdfConfiguration with properties set
+     */
+    public static AccumuloRdfConfigurationBuilder fromProperties(Properties props) {
+        AccumuloRdfConfigurationBuilder builder = new AccumuloRdfConfigurationBuilder() //
+                .setAuths(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_AUTHS, "")) //
+                .setRyaPrefix(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_RYA_PREFIX, "rya_"))//
+                .setVisibilities(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_VISIBILITIES, ""))
+                .setUseInference(
+                        getBoolean(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_INFERENCE, "false")))//
+                .setDisplayQueryPlan(getBoolean(
+                        props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_DISPLAY_QUERY_PLAN, "true")))//
+                .setAccumuloUser(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_USER)) //
+                .setAccumuloInstance(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_INSTANCE))//
+                .setAccumuloZooKeepers(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_ZOOKEEPERS))//
+                .setAccumuloPassword(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.ACCUMULO_PASSWORD))//
+                .setUseMockAccumulo(getBoolean(
+                        props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_MOCK_ACCUMULO, "false")))//
+                .setUseAccumuloPrefixHashing(getBoolean(
+                        props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_PREFIX_HASHING, "false")))//
+                .setUseCompositeCardinality(
+                        getBoolean(props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_COUNT_STATS, "false")))//
+                .setUseJoinSelectivity(getBoolean(
+                        props.getProperty(AbstractAccumuloRdfConfigurationBuilder.USE_JOIN_SELECTIVITY, "false")));
+        return builder;
+    }
+
+    @Override
+    protected AccumuloRdfConfigurationBuilder confBuilder() {
+        return this;
+    }
+
+    @Override
+    protected AccumuloRdfConfiguration createConf() {
+        return new AccumuloRdfConfiguration();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/accumulo.rya/src/test/java/org/apache/rya/accumulo/AccumuloRdfConfigurationTest.java
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/java/org/apache/rya/accumulo/AccumuloRdfConfigurationTest.java b/dao/accumulo.rya/src/test/java/org/apache/rya/accumulo/AccumuloRdfConfigurationTest.java
index e3f6396..02311b3 100644
--- a/dao/accumulo.rya/src/test/java/org/apache/rya/accumulo/AccumuloRdfConfigurationTest.java
+++ b/dao/accumulo.rya/src/test/java/org/apache/rya/accumulo/AccumuloRdfConfigurationTest.java
@@ -19,14 +19,16 @@ package org.apache.rya.accumulo;
  * under the License.
  */
 
-
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.security.Authorizations;
@@ -39,7 +41,7 @@ public class AccumuloRdfConfigurationTest {
 
     @Test
     public void testAuths() {
-        String[] arr = {"U", "FOUO"};
+        String[] arr = { "U", "FOUO" };
         String str = "U,FOUO";
         Authorizations auths = new Authorizations(arr);
 
@@ -72,4 +74,85 @@ public class AccumuloRdfConfigurationTest {
         assertEquals(setting, iteratorSettings[0]);
 
     }
+
+    @Test
+    public void testBuilder() {
+        String prefix = "rya_";
+        String auth = "U";
+        String visibility = "U";
+        String user = "user";
+        String password = "password";
+        String instance = "instance";
+        String zookeeper = "zookeeper";
+        boolean useMock = false;
+        boolean useComposite = true;
+        boolean usePrefixHash = true;
+        boolean useInference = true;
+        boolean displayPlan = false;
+
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration().getBuilder()//
+                .setAuths(auth)//
+                .setVisibilities(visibility)//
+                .setRyaPrefix(prefix)//
+                .setUseInference(useInference)//
+                .setUseCompositeCardinality(useComposite)//
+                .setDisplayQueryPlan(displayPlan)//
+                .setAccumuloInstance(instance)//
+                .setAccumuloPassword(password)//
+                .setAccumuloUser(user)//
+                .setAccumuloZooKeepers(zookeeper)//
+                .setUseMockAccumulo(useMock)//
+                .setUseAccumuloPrefixHashing(usePrefixHash)//
+                .build();
+
+        assertEquals(conf.getTablePrefix(), prefix);
+        assertEquals(conf.getCv(), visibility);
+        assertEquals(conf.getAuthorizations(), new Authorizations(auth));
+        assertEquals(conf.isInfer(), useInference);
+        assertEquals(conf.isUseCompositeCardinality(), useComposite);
+        assertEquals(conf.isDisplayQueryPlan(), displayPlan);
+        assertEquals(conf.getAccumuloInstance(), instance);
+        assertEquals(conf.getAccumuloPassword(), password);
+        assertEquals(conf.getAccumuloUser(), user);
+        assertEquals(conf.getAccumuloZookeepers(), zookeeper);
+        assertEquals(conf.getUseMockAccumulo(), useMock);
+        assertEquals(conf.isPrefixRowsWithHash(), usePrefixHash);
+
+    }
+
+    @Test
+    public void testBuilderFromProperties() throws FileNotFoundException, IOException {
+        String prefix = "rya_";
+        String auth = "U";
+        String visibility = "U";
+        String user = "user";
+        String password = "password";
+        String instance = "instance";
+        String zookeeper = "zookeeper";
+        boolean useMock = false;
+        boolean useComposite = true;
+        boolean usePrefixHash = true;
+        boolean useInference = true;
+        boolean displayPlan = false;
+
+        Properties props = new Properties();
+        props.load(new FileInputStream("src/test/resources/properties/rya.properties"));
+
+        AccumuloRdfConfiguration conf = AccumuloRdfConfiguration.fromProperties(props);
+
+        assertEquals(conf.getTablePrefix(), prefix);
+        assertEquals(conf.getCv(), visibility);
+        assertEquals(conf.getAuthorizations(), new Authorizations(auth));
+        assertEquals(conf.isInfer(), useInference);
+        assertEquals(conf.isUseCompositeCardinality(), useComposite);
+        assertEquals(conf.isDisplayQueryPlan(), displayPlan);
+        assertEquals(conf.getAccumuloInstance(), instance);
+        assertEquals(conf.getAccumuloPassword(), password);
+        assertEquals(conf.getAccumuloUser(), user);
+        assertEquals(conf.getAccumuloZookeepers(), zookeeper);
+        assertEquals(conf.getUseMockAccumulo(), useMock);
+        assertEquals(conf.isPrefixRowsWithHash(), usePrefixHash);
+
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/accumulo.rya/src/test/resources/properties/rya.properties
----------------------------------------------------------------------
diff --git a/dao/accumulo.rya/src/test/resources/properties/rya.properties b/dao/accumulo.rya/src/test/resources/properties/rya.properties
new file mode 100644
index 0000000..79a5cb4
--- /dev/null
+++ b/dao/accumulo.rya/src/test/resources/properties/rya.properties
@@ -0,0 +1,30 @@
+ # 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
+ 
+use.prefix.hashing true
+use.count.stats true
+use.join.selectivity true
+use.mock false
+use.display.plan false
+use.inference true
+accumulo.user user
+accumulo.password password
+accumulo.instance instance
+accumulo.zookeepers zookeeper
+accumulo.auths U
+accumulo.visibilities U
+accumuo.rya.prefix rya_

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
new file mode 100644
index 0000000..f50bf9a
--- /dev/null
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.mongodb;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.api.RdfCloudTripleStoreConfigurationBuilder;
+
+/**
+ * This builder class will set all of the core Mongo-backed Rya configuration
+ * parameters. Any builder extending this class will have setter methods for all
+ * of the necessary parameters to connect to a Mongo backed Rya instance.
+ *
+ * @param <B>
+ *            - builder returned by setter methods extending this class
+ * @param <C>
+ *            - configuration object constructed by the builder extending this
+ *            class
+ */
+public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractMongoDBRdfConfigurationBuilder<B, C>, C extends MongoDBRdfConfiguration>
+        extends RdfCloudTripleStoreConfigurationBuilder<B, C> {
+
+    private String user;
+    private String pass;
+    private boolean useMock = false;
+    private String host = "localhost";
+    private String port = DEFAULT_MONGO_PORT;
+    protected static final String DEFAULT_MONGO_PORT = "27017";
+    private String mongoCollectionPrefix = "rya_";
+    private String mongoDBName = "rya";
+
+    protected static final String MONGO_USER = "mongo.user";
+    protected static final String MONGO_PASSWORD = "mongo.password";
+    protected static final String MONGO_DB_NAME = "mongo.db.name";
+    protected static final String MONGO_COLLECTION_PREFIX = "mongo.collection.prefix";
+    protected static final String MONGO_HOST = "mongo.host";
+    protected static final String MONGO_PORT = "mongo.port";
+    protected static final String MONGO_AUTHS = "mongo.auths";
+    protected static final String MONGO_VISIBILITIES = "mongo.visibilities";
+    protected static final String MONGO_RYA_PREFIX = "mongo.rya.prefix";
+    protected static final String USE_INFERENCE = "use.inference";
+    protected static final String USE_DISPLAY_QUERY_PLAN = "use.display.plan";
+    protected static final String USE_MOCK_MONGO = "use.mock";
+
+    /**
+     * Sets Mongo user.
+     * 
+     * @param user - user name used to connect to Mongo
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoUser(String user) {
+        this.user = user;
+        return confBuilder();
+    }
+
+    /**
+     * Sets password for Mongo user specified by
+     * {@link AbstractMongoDBRdfConfigurationBuilder#setMongoUser(String)}.
+     * 
+     * @param password - password used to connect to Mongo
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoPassword(String password) {
+        this.pass = password;
+        return confBuilder();
+    }
+
+    /**
+     * Sets Mongo port. This parameter must be set to connect to an instance of
+     * MongoDB and will default to "27017" if no value is specified.
+     * 
+     * @param port - port used to connect Mongo
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoPort(String port) {
+        this.port = port;
+        return confBuilder();
+    }
+
+    /**
+     * Sets Mongo host. This parameter must be set to connect to an instance of
+     * MongoDB and will default to "localhost" if no value is specified.
+     * 
+     * @param host - host used to connect to Mongo
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoHost(String host) {
+        this.host = host;
+        return confBuilder();
+    }
+
+    /**
+     * Sets MongoDB name. This parameter must be set to connect to an instance
+     * of MongoDB and will default to "rya_triples" is no value is specified.
+     * 
+     * @param name - name of MongoDB to connect to
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoDBName(String name) {
+        this.mongoDBName = name;
+        return confBuilder();
+    }
+
+    /**
+     * Sets MongoDB Collection prefix. This parameter must be set to connect to
+     * an instance of MongoDB and will default to "rya_" is no value is
+     * specified.
+     * 
+     * @param name - name of Collection to connect to
+     * @return specified builder for chaining method invocations
+     */
+    public B setMongoCollectionPrefix(String prefix) {
+        this.mongoCollectionPrefix = prefix;
+        return confBuilder();
+    }
+
+    /**
+     * Set whether to use instance of embedded Mongo as backend for Rya
+     * instance.
+     * 
+     * @param useMock - indicates whether to use embedded Mongo as Rya backing
+     * @return specified builder for chaining method invocations
+     */
+    public B setUseMockMongo(boolean useMock) {
+        this.useMock = useMock;
+        return confBuilder();
+    }
+
+    /**
+     * @return extension of {@link MongoDBRdfConfiguration} with specified parameters set
+     */
+    public C build() {
+        return getConf(super.build());
+    }
+
+    /**
+     * Assigns builder values to appropriate parameters within the {@link Configuration} object.
+     * 
+     * @param conf - Configuration object
+     * @return - Configuration object with parameters set
+     */
+    private C getConf(C conf) {
+
+        conf.setUseMock(useMock);
+        conf.set("sc.useMongo", "true");
+        
+        if (user != null) {
+            conf.setMongoUser(user);
+        }
+        if (pass != null) {
+            conf.setMongoPassword(pass);
+        }
+        conf.setMongoDBName(mongoDBName);
+        conf.setCollectionName(mongoCollectionPrefix);
+        conf.setTablePrefix(mongoCollectionPrefix);
+        conf.setMongoInstance(host);
+        conf.setMongoPort(port);
+
+        return conf;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
index 49e74cc..e88abee 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
@@ -19,15 +19,13 @@ package org.apache.rya.mongodb;
  * under the License.
  */
 
-
-
 import java.util.List;
-
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
-import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
+import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.mongodb.MongoClient;
 
@@ -38,9 +36,10 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
     public static final String MONGO_DB_NAME = "mongo.db.name";
     public static final String MONGO_COLLECTION_PREFIX = "mongo.db.collectionprefix";
     public static final String MONGO_USER = "mongo.db.user";
-    public static final String  MONGO_USER_PASSWORD = "mongo.db.userpassword";
+    public static final String MONGO_USER_PASSWORD = "mongo.db.userpassword";
     public static final String CONF_ADDITIONAL_INDEXERS = "ac.additional.indexers";
-	private MongoClient mongoClient;
+    public static final String USE_MOCK_MONGO = ".useMockInstance";
+    private MongoClient mongoClient;
 
     public MongoDBRdfConfiguration() {
         super();
@@ -50,70 +49,193 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
         super(other);
     }
 
+    /**
+     * Creates a MongoRdfConfiguration object from a Properties file. This
+     * method assumes that all values in the Properties file are Strings and
+     * that the Properties file uses the keys below.
+     * 
+     * <br>
+     * <ul>
+     * <li>"mongo.auths" - String of Mongo authorizations.  Empty auths used by default.
+     * <li>"mongo.visibilities" - String of Mongo visibilities assigned to ingested triples.
+     * <li>"mongo.user" - Mongo user.  Empty by default.
+     * <li>"mongo.password" - Mongo password.  Empty by default.
+     * <li>"mongo.host" - Mongo host.  Default host is "localhost"
+     * <li>"mongo.port" - Mongo port.  Default port is "27017".
+     * <li>"mongo.db.name" - Name of MongoDB.  Default name is "rya_triples".
+     * <li>"mongo.collection.prefix" - Mongo collection prefix. Default is "rya_".
+     * <li>"mongo.rya.prefix" - Prefix for Mongo Rya instance.  Same as value of "mongo.collection.prefix".
+     * <li>"use.mock" - Use a Embedded Mongo instance as back-end for Rya instance. False by default.
+     * <li>"use.display.plan" - Display query plan during evaluation. Useful for debugging.  True by default.
+     * <li>"use.inference" - Use backward chaining inference during query.  False by default.
+     * </ul>
+     * <br>
+     * 
+     * @param props
+     *            - Properties file containing Mongo specific configuration
+     *            parameters
+     * @return MongoRdfConfiguration with properties set
+     */
+    public static MongoDBRdfConfiguration fromProperties(Properties props) {
+        return MongoDBRdfConfigurationBuilder.fromProperties(props);
+    }
+
+    public MongoDBRdfConfigurationBuilder getBuilder() {
+        return new MongoDBRdfConfigurationBuilder();
+    }
+
     @Override
     public MongoDBRdfConfiguration clone() {
         return new MongoDBRdfConfiguration(this);
     }
-
+    /**
+     * @return name of Mongo Collection containing Rya triples
+     */
     public String getTriplesCollectionName() {
         return this.get(MONGO_COLLECTION_PREFIX, "rya") + "_triples";
     }
-
+    
+    /**
+     * @return name of Mongo Collection 
+     */
     public String getCollectionName() {
         return this.get(MONGO_COLLECTION_PREFIX, "rya");
     }
 
+    /**
+     * Sets Mongo Collection name
+     * @param name - name of Mongo Collection to connect to
+     */
     public void setCollectionName(String name) {
+        Preconditions.checkNotNull(name);
         this.set(MONGO_COLLECTION_PREFIX, name);
     }
 
+    /**
+     * @return name of Mongo Host
+     */
     public String getMongoInstance() {
         return this.get(MONGO_INSTANCE, "localhost");
     }
 
+    /**
+     * Sets name of Mongo Host
+     * @param name - name of Mongo Host to connect to
+     */
     public void setMongoInstance(String name) {
+        Preconditions.checkNotNull(name);
         this.set(MONGO_INSTANCE, name);
     }
 
+    /**
+     * @return port that Mongo is running on
+     */
     public String getMongoPort() {
-        return this.get(MONGO_INSTANCE_PORT, "27017");
+        return this.get(MONGO_INSTANCE_PORT, AbstractMongoDBRdfConfigurationBuilder.DEFAULT_MONGO_PORT);
     }
 
+    /**
+     * Sets port that Mongo will run on
+     * @param name - Mongo port to connect to
+     */
     public void setMongoPort(String name) {
+        Preconditions.checkNotNull(name);
         this.set(MONGO_INSTANCE_PORT, name);
     }
 
+    /**
+     * @return name of MongoDB
+     */
     public String getMongoDBName() {
         return this.get(MONGO_DB_NAME, "rya");
     }
 
+    /**
+     * Sets name of MongoDB
+     * @param name - name of MongoDB to connect to
+     */
     public void setMongoDBName(String name) {
+        Preconditions.checkNotNull(name);
         this.set(MONGO_DB_NAME, name);
     }
 
+    /**
+     * Tells Rya to use an embedded Mongo instance as its backing
+     * if set to true.  By default this is set to false.
+     * @param useMock
+     */
+    public void setUseMock(boolean useMock) {
+        this.setBoolean(USE_MOCK_MONGO, useMock);
+    }
+    
+    /**
+     * Get whether an embedded Mongo is being used as the backing
+     * for Rya.
+     * @return true if embedded Mongo is being used, and false otherwise
+     */
+    public boolean getUseMock() {
+        return getBoolean(USE_MOCK_MONGO, false);
+    }
+
+    /**
+     * @return name of NameSpace Mongo Collection
+     */
     public String getNameSpacesCollectionName() {
         return this.get(MONGO_COLLECTION_PREFIX, "rya") + "_ns";
     }
-    
+
+    /**
+     * Sets name of Mongo User
+     * @param user - name of Mongo user to connect to
+     */
+    public void setMongoUser(String user) {
+        Preconditions.checkNotNull(user);
+        set(MONGO_USER, user);
+    }
+
+    /**
+     * @return name of Mongo user
+     */
+    public String getMongoUser() {
+        return get(MONGO_USER);
+    }
+
+    /**
+     * Sets Mongo password
+     * @param password - password to connect to Mongo
+     */
+    public void setMongoPassword(String password) {
+        Preconditions.checkNotNull(password);
+        set(MONGO_USER_PASSWORD, password);
+    }
+
+    /**
+     * @return Mongo password
+     */
+    public String getMongoPassword() {
+        return get(MONGO_USER_PASSWORD);
+    }
+
     public void setAdditionalIndexers(Class<? extends MongoSecondaryIndex>... indexers) {
         List<String> strs = Lists.newArrayList();
-        for (Class<?> ai : indexers){
+        for (Class<?> ai : indexers) {
             strs.add(ai.getName());
         }
-        
-        setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
+
+        setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[] {}));
     }
 
     public List<MongoSecondaryIndex> getAdditionalIndexers() {
         return getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
-    }    
-    
-    public void setMongoClient(MongoClient client){
-    	this.mongoClient = client;
     }
-    
+
+    public void setMongoClient(MongoClient client) {
+        Preconditions.checkNotNull(client);
+        this.mongoClient = client;
+    }
+
     public MongoClient getMongoClient() {
-    	return mongoClient;
+        return mongoClient;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfigurationBuilder.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfigurationBuilder.java
new file mode 100644
index 0000000..cbe8f45
--- /dev/null
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfigurationBuilder.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.mongodb;
+
+import java.util.Properties;
+
+/**
+ * This is a concrete extension of the
+ * {@link AbstractMongoDBRdfConfigurationBuilder} class which builds an
+ * {@link MongoDBRdfConfiguration} object. This builder creates an
+ * MongoDBRdfConfiguratio object and sets all of the parameters required to
+ * connect to an Mongo Rya instance.
+ *
+ */
+public class MongoDBRdfConfigurationBuilder
+        extends AbstractMongoDBRdfConfigurationBuilder<MongoDBRdfConfigurationBuilder, MongoDBRdfConfiguration> {
+
+    /**
+     * Creates a MongoRdfConfiguration object from a Properties file. This
+     * method assumes that all values in the Properties file are Strings and
+     * that the Properties file uses the keys below.
+     * 
+     * <br>
+     * <ul>
+     * <li>"mongo.auths" - String of Mongo authorizations. Empty auths used by
+     * default.
+     * <li>"mongo.visibilities" - String of Mongo visibilities assigned to
+     * ingested triples.
+     * <li>"mongo.user" - Mongo user. Empty by default.
+     * <li>"mongo.password" - Mongo password. Empty by default.
+     * <li>"mongo.host" - Mongo host. Default host is "localhost"
+     * <li>"mongo.port" - Mongo port. Default port is "27017".
+     * <li>"mongo.db.name" - Name of MongoDB. Default name is "rya_triples".
+     * <li>"mongo.collection.prefix" - Mongo collection prefix. Default is
+     * "rya_".
+     * <li>"mongo.rya.prefix" - Prefix for Mongo Rya instance. Same as value of
+     * "mongo.collection.prefix".
+     * <li>"use.mock" - Use a Embedded Mongo instance as back-end for Rya
+     * instance. False by default.
+     * <li>"use.display.plan" - Display query plan during evaluation. Useful for
+     * debugging. True by default.
+     * <li>"use.inference" - Use backward chaining inference during query. False
+     * by default.
+     * </ul>
+     * <br>
+     * 
+     * @param props
+     *            - Properties file containing Mongo specific configuration
+     *            parameters
+     * @return MongoRdfConfiguration with properties set
+     */
+    public static MongoDBRdfConfiguration fromProperties(Properties props) {
+        try {
+
+            MongoDBRdfConfigurationBuilder builder = new MongoDBRdfConfigurationBuilder() //
+                    .setAuths(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_AUTHS, "")) //
+                    .setRyaPrefix(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_RYA_PREFIX, "rya_"))//
+                    .setVisibilities(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_VISIBILITIES, ""))
+                    .setUseInference(getBoolean(
+                            props.getProperty(AbstractMongoDBRdfConfigurationBuilder.USE_INFERENCE, "false")))//
+                    .setDisplayQueryPlan(getBoolean(
+                            props.getProperty(AbstractMongoDBRdfConfigurationBuilder.USE_DISPLAY_QUERY_PLAN, "true")))//
+                    .setMongoUser(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_USER)) //
+                    .setMongoPassword(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_PASSWORD))//
+                    .setMongoCollectionPrefix(
+                            props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_COLLECTION_PREFIX, "rya_"))//
+                    .setMongoDBName(
+                            props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_DB_NAME, "rya_triples"))//
+                    .setMongoHost(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_HOST, "localhost"))//
+                    .setMongoPort(props.getProperty(AbstractMongoDBRdfConfigurationBuilder.MONGO_PORT,
+                            AbstractMongoDBRdfConfigurationBuilder.DEFAULT_MONGO_PORT))//
+                    .setUseMockMongo(getBoolean(
+                            props.getProperty(AbstractMongoDBRdfConfigurationBuilder.USE_MOCK_MONGO, "false")));
+
+            return builder.build();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected MongoDBRdfConfigurationBuilder confBuilder() {
+        return this;
+    }
+
+    @Override
+    protected MongoDBRdfConfiguration createConf() {
+        return new MongoDBRdfConfiguration();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
new file mode 100644
index 0000000..c047fde
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.mongodb;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Properties;
+
+import org.junit.Test;
+
+public class MongoDBRdfConfigurationTest {
+
+    @Test
+    public void testBuilder() {
+        String prefix = "prefix_";
+        String auth = "U,V,W";
+        String visibility = "U,W";
+        String user = "user";
+        String password = "password";
+        boolean useMock = true;
+        boolean useInference = true;
+        boolean displayPlan = false;
+
+        MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration().getBuilder()
+                .setVisibilities(visibility)
+                .setUseInference(useInference)
+                .setDisplayQueryPlan(displayPlan)
+                .setUseMockMongo(useMock)
+                .setMongoCollectionPrefix(prefix)
+                .setMongoDBName("dbname")
+                .setMongoHost("host")
+                .setMongoPort("1000")
+                .setAuths(auth)
+                .setMongoUser(user)
+                .setMongoPassword(password).build();
+
+        assertEquals(conf.getTablePrefix(), prefix);
+        assertTrue(Arrays.equals(conf.getAuths(), new String[] { "U", "V", "W" }));
+        assertEquals(conf.getCv(), visibility);
+        assertEquals(conf.isInfer(), useInference);
+        assertEquals(conf.isDisplayQueryPlan(), displayPlan);
+        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
+        assertEquals(conf.getMongoPort(), "1000");
+        assertEquals(conf.getMongoDBName(), "dbname");
+        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
+        assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
+
+    }
+
+    @Test
+    public void testBuilderFromProperties() throws FileNotFoundException, IOException {
+        String prefix = "prefix_";
+        String auth = "U";
+        String visibility = "U";
+        String user = "user";
+        String password = "password";
+        boolean useMock = true;
+        boolean useInference = true;
+        boolean displayPlan = false;
+
+        Properties props = new Properties();
+        props.load(new FileInputStream("src/test/resources/rya.properties"));
+
+        MongoDBRdfConfiguration conf = MongoDBRdfConfiguration.fromProperties(props);
+
+        assertEquals(conf.getTablePrefix(), prefix);
+        assertTrue(Arrays.equals(conf.getAuths(), new String[] { auth }));
+        assertEquals(conf.getCv(), visibility);
+        assertEquals(conf.isInfer(), useInference);
+        assertEquals(conf.isDisplayQueryPlan(), displayPlan);
+        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
+        assertEquals(conf.getMongoPort(), "1000");
+        assertEquals(conf.getMongoDBName(), "dbname");
+        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
+        assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/dao/mongodb.rya/src/test/resources/rya.properties
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/resources/rya.properties b/dao/mongodb.rya/src/test/resources/rya.properties
new file mode 100644
index 0000000..8c37dc3
--- /dev/null
+++ b/dao/mongodb.rya/src/test/resources/rya.properties
@@ -0,0 +1,29 @@
+ # Licensed to the Apache Software Foundation (ASF) under one
+ # or more contributor license agreements.  See the NOTICE file
+ # distributed with this work for additional information
+ # regarding copyright ownership.  The ASF licenses this file
+ # to you under the Apache License, Version 2.0 (the
+ # "License"); you may not use this file except in compliance
+ # with the License.  You may obtain a copy of the License at
+ #
+ #   http://www.apache.org/licenses/LICENSE-2.0
+ #
+ # Unless required by applicable law or agreed to in writing,
+ # software distributed under the License is distributed on an
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ # KIND, either express or implied.  See the License for the
+ # specific language governing permissions and limitations
+ # under the License
+
+use.mock true
+use.display.plan false
+use.inference true
+mongo.user user
+mongo.password password
+mongo.host host
+mongo.port 1000
+mongo.auths U
+mongo.visibilities U
+mongo.db.name dbname
+mongo.collection.prefix prefix_
+

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/api/client/accumulo/AccumuloInstall.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/api/client/accumulo/AccumuloInstall.java b/extras/indexing/src/main/java/org/apache/rya/api/client/accumulo/AccumuloInstall.java
index 1849fd8..6fd0d36 100644
--- a/extras/indexing/src/main/java/org/apache/rya/api/client/accumulo/AccumuloInstall.java
+++ b/extras/indexing/src/main/java/org/apache/rya/api/client/accumulo/AccumuloInstall.java
@@ -205,6 +205,7 @@ public class AccumuloInstall extends AccumuloCommand implements Install {
         final PrecomputedJoinUpdaterType updaterType = fluoHolder.isPresent() ? PrecomputedJoinUpdaterType.FLUO : PrecomputedJoinUpdaterType.NO_UPDATE;
         conf.set(ConfigUtils.PCJ_UPDATER_TYPE, updaterType.toString());
 
+
         // XXX The Accumulo implementation of the secondary indices make need all
         //     of the accumulo connector's parameters to initialize themselves, so
         //     we need to include them here. It would be nice if the secondary