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:16 UTC

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

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfiguration.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfiguration.java b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfiguration.java
new file mode 100644
index 0000000..a6a8c30
--- /dev/null
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfiguration.java
@@ -0,0 +1,658 @@
+/*
+ * 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.indexing.accumulo;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.accumulo.AbstractAccumuloRdfConfigurationBuilder;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.accumulo.AccumuloRdfConfigurationBuilder;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.indexing.accumulo.entity.EntityCentricIndex;
+import org.apache.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
+import org.apache.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
+import org.apache.rya.indexing.external.PrecomputedJoinIndexer;
+import org.openrdf.sail.Sail;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * This class is an extension of the AccumuloRdfConfiguration object used to to
+ * create a {@link Sail} connection to an Accumulo backed instance of Rya. This
+ * configuration object is designed to create Accumulo Rya Sail connections
+ * where one or more of the Accumulo Rya Indexes are enabled. These indexes
+ * include the {@link AccumuloFreeTextIndexer}, {@link AccumuloTemporalIndexer},
+ * {@link EntityCentricIndex}, and the {@link PrecomputedJoinIndexer}.
+ *
+ */
+public class AccumuloIndexingConfiguration extends AccumuloRdfConfiguration {
+
+    private AccumuloIndexingConfiguration() {
+    }
+
+    public static AccumuloIndexingConfigBuilder builder() {
+        return new AccumuloIndexingConfigBuilder();
+    }
+
+    /**
+     * Creates an AccumuloIndexingConfiguration 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>"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.
+     * <li>"use.freetext" - Use Accumulo Freetext Indexer for querying and
+     * ingest. Default is false.
+     * <li>"use.temporal" - Use Accumulo Temporal Indexer for querying and
+     * ingest. Default is false.
+     * <li>"use.entity" - Use Accumulo Entity Index for querying and ingest.
+     * Default is false.
+     * <li>"fluo.app.name" - Set name of Fluo App to update PCJs.
+     * <li>"use.pcj" - Use PCJs for query optimization. Default is false.
+     * <li>"use.optimal.pcj" - Use optimal PCJ for query optimization. Default
+     * is false.
+     * <li>"pcj.tables" - PCJ tables to be used, specified as comma delimited
+     * Strings with no spaces between. If no tables are specified, all
+     * registered tables are used.
+     * <li>"freetext.predicates" - Freetext predicates used for ingest. Specify
+     * as comma delimited Strings with no spaces between. Empty by default.
+     * <li>"temporal.predicates" - Temporal predicates used for ingest. Specify
+     * as comma delimited Strings with no spaces between. Empty by default.
+     * </ul>
+     * <br>
+     * 
+     * @param props
+     *            - Properties file containing Accumulo specific configuration
+     *            parameters
+     * @return AccumumuloIndexingConfiguration with properties set
+     */
+    public static AccumuloIndexingConfiguration fromProperties(Properties props) {
+        return AccumuloIndexingConfigBuilder.fromProperties(props);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link EntitCentricIndex} for ingest and at
+     * query time. The default value is false, and if useEntity is set to true
+     * and the EntityIndex does not exist, then useEntity will default to false.
+     * 
+     * @param useEntity
+     *            - use entity indexing
+     */
+    public void setUseEntity(boolean useEntity) {
+        setBoolean(ConfigUtils.USE_ENTITY, useEntity);
+    }
+
+    /**
+     * @return boolean indicating whether or not {@link EntityCentricIndex} is enabled
+     */
+    public boolean getUseEntity() {
+        return getBoolean(ConfigUtils.USE_ENTITY, false);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link AccumuloTemproalIndexer} for ingest and
+     * at query time. The default value is false, and if useTemporal is set to
+     * true and the TemporalIndex does not exist, then useTemporal will default
+     * to false.
+     * 
+     * @param useTemporal
+     *            - use temporal indexing
+     */
+    public void setUseTemporal(boolean useTemporal) {
+        setBoolean(ConfigUtils.USE_TEMPORAL, useTemporal);
+    }
+
+    /**
+     * @return boolean indicating whether or not {@link AccumuloTemporalIndexer} is enabled
+     */
+    public boolean getUseTemporal() {
+        return getBoolean(ConfigUtils.USE_TEMPORAL, false);
+    }
+
+    /**
+     * @return boolean indicating whether or not {@link AccumuloFreeTextIndexer} is enabled
+     */
+    public boolean getUseFreetext() {
+        return getBoolean(ConfigUtils.USE_FREETEXT, false);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link AccumuloFreeTextIndexer} for ingest and
+     * at query time. The default value is false, and if useFreeText is set to
+     * true and the FreeTextIndex does not exist, then useFreeText will default
+     * to false.
+     * 
+     * @param useFreeText
+     *            - use freetext indexing
+     */
+    public void setUseFreetext(boolean useFreetext) {
+        setBoolean(ConfigUtils.USE_FREETEXT, useFreetext);
+    }
+
+    /**
+     * @return boolean indicating whether or not {@link PrecomputedJoinIndexer} is enabled
+     */
+    public boolean getUsePCJUpdater() {
+        return getBoolean(ConfigUtils.USE_PCJ_UPDATER_INDEX, false);
+    }
+
+    public void setUsePCJUpdater(boolean usePCJUpdater) {
+        setBoolean(ConfigUtils.USE_PCJ_UPDATER_INDEX, usePCJUpdater);
+        if (usePCJUpdater) {
+            set(ConfigUtils.PCJ_STORAGE_TYPE, "ACCUMULO");
+            set(ConfigUtils.PCJ_UPDATER_TYPE, "FLUO");
+        }
+    }
+
+    /**
+     * 
+     * Specify the name of the PCJ Fluo updater application. A non-null
+     * application results in the {@link PrecomputedJoinIndexer} being activated
+     * so that all triples ingested into Rya are also ingested into Fluo to
+     * update any registered PCJs. PreomputedJoinIndexer is turned off by
+     * default. If no fluo application of the specified name exists, a
+     * RuntimeException will occur.
+     * 
+     * @param fluoAppName
+     *            - use entity indexing
+     */
+    public void setFluoAppUpdaterName(String fluoAppName) {
+        Preconditions.checkNotNull(fluoAppName, "Fluo app name cannot be null.");
+        setUsePCJUpdater(true);
+        set(ConfigUtils.FLUO_APP_NAME, fluoAppName);
+    }
+
+    /**
+     * @return name of the Fluo PCJ Updater application
+     */
+    public String getFluoAppUpdaterName() {
+        return get(ConfigUtils.FLUO_APP_NAME);
+    }
+
+    /**
+     * Use Precomputed Joins as a query optimization.
+     * 
+     * @param usePcj
+     *            - use PCJ
+     */
+    public void setUsePCJ(boolean usePCJ) {
+        setBoolean(ConfigUtils.USE_PCJ, usePCJ);
+    }
+
+    /**
+     * @return boolean indicating whether or not PCJs are enabled for querying
+     */
+    public boolean getUsePCJ() {
+        return getBoolean(ConfigUtils.USE_PCJ, false);
+    }
+
+    /**
+     * Use Precomputed Joins as a query optimization and attempt to find the
+     * best combination of PCJ in the query plan
+     * 
+     * @param useOptimalPcj
+     *            - use optimal pcj plan
+     */
+    public void setUseOptimalPCJ(boolean useOptimalPCJ) {
+        setBoolean(ConfigUtils.USE_OPTIMAL_PCJ, useOptimalPCJ);
+    }
+
+    /**
+     * @return boolean indicating whether or not query planner will look for optimal
+     * combinations of PCJs when forming the query plan.
+     */
+    public boolean getUseOptimalPCJ() {
+        return getBoolean(ConfigUtils.USE_OPTIMAL_PCJ, false);
+    }
+
+    /**
+     * Sets the predicates used for freetext indexing
+     * @param predicates - Array of predicate URI strings used for freetext indexing
+     */
+    public void setAccumuloFreeTextPredicates(String[] predicates) {
+        Preconditions.checkNotNull(predicates, "Freetext predicates cannot be null.");
+        setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, predicates);
+    }
+
+    /**
+     * Gets the predicates used for freetext indexing
+     * @return Array of predicate URI strings used for freetext indexing
+     */
+    public String[] getAccumuloFreeTextPredicates() {
+        return getStrings(ConfigUtils.FREETEXT_PREDICATES_LIST);
+    }
+
+    /**
+     * Sets the predicates used for temporal indexing
+     * @param predicates - Array of predicate URI strings used for temporal indexing
+     */
+    public void setAccumuloTemporalPredicates(String[] predicates) {
+        Preconditions.checkNotNull(predicates, "Freetext predicates cannot be null.");
+        setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, predicates);
+    }
+
+    /**
+     * Gets the predicates used for temporal indexing
+     * @return Array of predicate URI strings used for temporal indexing
+     */
+    public String[] getAccumuloTemporalPredicates() {
+        return getStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST);
+    }
+    
+    private static Set<RyaURI> getPropURIFromStrings(String ... props) {
+        Set<RyaURI> properties = new HashSet<>();
+        for(String prop: props) {
+            properties.add(new RyaURI(prop));
+        }
+        return properties;
+    }
+    
+    /**
+     * Concrete extension of {@link AbstractAccumuloRdfConfigurationBuilder}
+     * that adds setter methods to configure Accumulo Rya Indexers in addition
+     * the core Accumulo Rya configuration. This builder should be used instead
+     * of {@link AccumuloRdfConfigurationBuilder} to configure a query client to
+     * use one or more Accumulo Indexers.
+     *
+     */
+    public static class AccumuloIndexingConfigBuilder extends
+            AbstractAccumuloRdfConfigurationBuilder<AccumuloIndexingConfigBuilder, AccumuloIndexingConfiguration> {
+
+        private String fluoAppName;
+        private boolean useFreetext = false;
+        private boolean useTemporal = false;
+        private boolean useEntity = false;
+        private boolean useMetadata = false;
+        private String[] freetextPredicates;
+        private String[] temporalPredicates;
+        private boolean usePcj = false;
+        private boolean useOptimalPcj = false;
+        private String[] pcjs = new String[0];
+        private Set<RyaURI> metadataProps = new HashSet<>();
+
+        private static final String USE_FREETEXT = "use.freetext";
+        private static final String USE_TEMPORAL = "use.temporal";
+        private static final String USE_ENTITY = "use.entity";
+        private static final String FLUO_APP_NAME = "fluo.app.name";
+        private static final String USE_PCJ = "use.pcj";
+        private static final String USE_OPTIMAL_PCJ = "use.optimal.pcj";
+        private static final String TEMPORAL_PREDICATES = "temporal.predicates";
+        private static final String FREETEXT_PREDICATES = "freetext.predicates";
+        private static final String PCJ_TABLES = "pcj.tables";
+        private static final String USE_STATEMENT_METADATA = "use.metadata";
+        private static final String STATEMENT_METADATA_PROPERTIES = "metadata.properties";
+
+        /**
+         * Creates an AccumuloIndexingConfiguration 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>"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.
+         * <li>"use.freetext" - Use Accumulo Freetext Indexer for querying and
+         * ingest. Default is false.
+         * <li>"use.temporal" - Use Accumulo Temporal Indexer for querying and
+         * ingest. Default is false.
+         * <li>"use.entity" - Use Accumulo Entity Index for querying and ingest.
+         * Default is false.
+         * <li>"use.metadata" - Use Accumulo StatementMetadata index for querying Statement Properties.
+         * Default is false.
+         * <li>"metadata.properties" - Set Statement Properties that can be queried using the StatementMetadataOptimizer.
+         * Default is empty.
+         * <li>"fluo.app.name" - Set name of Fluo App to update PCJs
+         * <li>"use.pcj" - Use PCJs for query optimization. Default is false.
+         * <li>"use.optimal.pcj" - Use optimal PCJ for query optimization.
+         * Default is false.
+         * <li>"pcj.tables" - PCJ tables to be used, specified as comma
+         * delimited Strings with no spaces between. If no tables are specified,
+         * all registered tables are used.
+         * <li>"freetext.predicates" - Freetext predicates used for ingest.
+         * Specify as comma delimited Strings with no spaces between. Empty by
+         * default.
+         * <li>"temporal.predicates" - Temporal predicates used for ingest.
+         * Specify as comma delimited Strings with no spaces between. Empty by
+         * default.
+         * </ul>
+         * <br>
+         * 
+         * @param props
+         *            - Properties file containing Accumulo specific
+         *            configuration parameters
+         * @return AccumumuloIndexingConfiguration with properties set
+         */
+        public static AccumuloIndexingConfiguration fromProperties(Properties props) {
+            Preconditions.checkNotNull(props);
+            try {
+                AccumuloIndexingConfigBuilder builder = new AccumuloIndexingConfigBuilder() //
+                        .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")))//
+                        .setUseAccumuloFreetextIndex(getBoolean(props.getProperty(USE_FREETEXT, "false")))//
+                        .setUseAccumuloTemporalIndex(getBoolean(props.getProperty(USE_TEMPORAL, "false")))//
+                        .setUseAccumuloEntityIndex(getBoolean(props.getProperty(USE_ENTITY, "false")))//
+                        .setAccumuloFreeTextPredicates(props.getProperty(FREETEXT_PREDICATES))//
+                        .setAccumuloTemporalPredicates(props.getProperty(TEMPORAL_PREDICATES))//
+                        .setUsePcj(getBoolean(props.getProperty(USE_PCJ, "false")))//
+                        .setUseOptimalPcj(getBoolean(props.getProperty(USE_OPTIMAL_PCJ, "false")))//
+                        .setPcjTables(props.getProperty(PCJ_TABLES))//
+                        .setPcjUpdaterFluoAppName(props.getProperty(FLUO_APP_NAME))
+                        .setUseStatementMetadata(getBoolean(props.getProperty(USE_STATEMENT_METADATA)))
+                        .setStatementMetadataProperties(getPropURIFromStrings(props.getProperty(STATEMENT_METADATA_PROPERTIES)));
+
+                return builder.build();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * 
+         * Specify whether to use use {@link AccumuloFreeTextIndexer} for ingest
+         * and at query time. The default value is false, and if useFreeText is
+         * set to true and the FreeTextIndex does not exist, then useFreeText
+         * will default to false.
+         * 
+         * @param useFreeText
+         *            - use freetext indexing
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUseAccumuloFreetextIndex(boolean useFreeText) {
+            this.useFreetext = useFreeText;
+            return this;
+        }
+
+        /**
+         * 
+         * Specify whether to use use {@link AccumuloTemporalIndexer} for ingest
+         * and at query time. The default value is false, and if useTemporal is
+         * set to true and the TemporalIndex does not exist, then useTemporal
+         * will default to false.
+         * 
+         * @param useTemporal
+         *            - use temporal indexing
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUseAccumuloTemporalIndex(boolean useTemporal) {
+            this.useTemporal = useTemporal;
+            return this;
+        }
+
+        /**
+         * 
+         * Specify whether to use use {@link EntitCentricIndex} for ingest and
+         * at query time. The default value is false, and if useEntity is set to
+         * true and the EntityIndex does not exist, then useEntity will default
+         * to false.
+         * 
+         * @param useEntity
+         *            - use entity indexing
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUseAccumuloEntityIndex(boolean useEntity) {
+            this.useEntity = useEntity;
+            return this;
+        }
+
+        /**
+         * 
+         * Specify the name of the PCJ Fluo updater application. A non-null
+         * application results in the {@link PrecomputedJoinIndexer} being
+         * activated so that all triples ingested into Rya are also ingested
+         * into Fluo to update any registered PCJs. PreomputedJoinIndexer is
+         * turned off by default. If no fluo application of the specified name
+         * exists, a RuntimeException will be thrown.
+         * 
+         * @param fluoAppName
+         *            - use entity indexing
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setPcjUpdaterFluoAppName(String fluoAppName) {
+            this.fluoAppName = fluoAppName;
+            return this;
+        }
+
+        /**
+         * 
+         * @param predicates
+         *            - String of comma delimited predicates used by the
+         *            FreetextIndexer to determine which triples to index
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setAccumuloFreeTextPredicates(String... predicates) {
+            this.freetextPredicates = predicates;
+            return this;
+        }
+
+        /**
+         * 
+         * @param predicates
+         *            - String of comma delimited predicates used by the
+         *            TemporalIndexer to determine which triples to index
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setAccumuloTemporalPredicates(String... predicates) {
+            this.temporalPredicates = predicates;
+            return this;
+        }
+
+        /**
+         * Use Precomputed Joins as a query optimization.
+         * 
+         * @param usePcj
+         *            - use PCJ
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUsePcj(boolean usePcj) {
+            this.usePcj = usePcj;
+            return this;
+        }
+
+        /**
+         * Use Precomputed Joins as a query optimization and attempt to find the
+         * best combination of PCJs in the query plan
+         * 
+         * @param useOptimalPcj
+         *            - use optimal pcj plan
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUseOptimalPcj(boolean useOptimalPcj) {
+            this.useOptimalPcj = useOptimalPcj;
+            return this;
+        }
+
+        /**
+         * Specify a collection of PCJ tables to use for query optimization. If
+         * no tables are specified and PCJs are enabled for query evaluation,
+         * then all registered PCJs will be considered when optimizing the
+         * query.
+         * 
+         * @param pcjs
+         *            - array of PCJs to be used for query evaluation
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setPcjTables(String... pcjs) {
+            this.pcjs = pcjs;
+            return this;
+        }
+        
+        /**
+         * Specify whether or not to use {@link StatementMetadataOptimizer} to query on Statement
+         * properties.
+         * @param useMetadata
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setUseStatementMetadata(boolean useMetadata) {
+            this.useMetadata = useMetadata;
+            return this;
+        }
+        
+        /**
+         * Specify properties that the {@link StatementMetadataOptimizer} will use to query
+         * @param useMetadata
+         * @return AccumuloIndexingConfigBuilder for chaining method invocations
+         */
+        public AccumuloIndexingConfigBuilder setStatementMetadataProperties(Set<RyaURI> metadataProps) {
+            this.metadataProps = metadataProps;
+            return this;
+        }
+        
+        
+        /**
+         * @return {@link AccumuloIndexingConfiguration} object with specified parameters set
+         */
+        public AccumuloIndexingConfiguration build() {
+            AccumuloIndexingConfiguration conf = getConf(super.build());
+
+            return conf;
+        }
+
+        /**
+         * Assigns builder values to appropriate parameters within the {@link Configuration} object.
+         * 
+         * @param conf - Configuration object
+         * @return - Configuration object with parameters set
+         */
+        private AccumuloIndexingConfiguration getConf(AccumuloIndexingConfiguration conf) {
+
+            Preconditions.checkNotNull(conf);
+
+            if (fluoAppName != null) {
+                conf.setFluoAppUpdaterName(fluoAppName);
+            }
+            if (useFreetext) {
+                conf.setUseFreetext(useFreetext);
+                if (freetextPredicates != null) {
+                    conf.setAccumuloFreeTextPredicates(freetextPredicates);
+                }
+            }
+            if (useTemporal) {
+                conf.setUseTemporal(useTemporal);
+                if (temporalPredicates != null) {
+                    conf.setAccumuloTemporalPredicates(temporalPredicates);
+                }
+            }
+
+            if (usePcj || useOptimalPcj) {
+                conf.setUsePCJ(usePcj);
+                conf.setUseOptimalPCJ(useOptimalPcj);
+                if (pcjs.length > 1 || (pcjs.length == 1 && pcjs[0] != null)) {
+                    conf.setPcjTables(Lists.newArrayList(pcjs));
+                }
+            }
+            
+            if(useMetadata) {
+                conf.setUseStatementMetadata(useMetadata);
+                conf.setStatementMetadataProperties(metadataProps);
+            }
+
+            conf.setBoolean(ConfigUtils.USE_ENTITY, useEntity);
+
+            return conf;
+        }
+
+        @Override
+        protected AccumuloIndexingConfigBuilder confBuilder() {
+            return this;
+        }
+
+        @Override
+        protected AccumuloIndexingConfiguration createConf() {
+            return new AccumuloIndexingConfiguration();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/ConfigUtils.java b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/ConfigUtils.java
index 7f10629..17438b1 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/ConfigUtils.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/ConfigUtils.java
@@ -57,6 +57,7 @@ import org.apache.rya.indexing.entity.EntityIndexOptimizer;
 import org.apache.rya.indexing.entity.update.mongo.MongoEntityIndexer;
 import org.apache.rya.indexing.external.PrecomputedJoinIndexer;
 import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
+import org.apache.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
 import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
 import org.apache.rya.indexing.statement.metadata.matching.StatementMetadataOptimizer;
 import org.openrdf.model.URI;
@@ -66,20 +67,51 @@ import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 
 /**
- * A set of configuration utils to read a Hadoop {@link Configuration} object and create Cloudbase/Accumulo objects.
- * Soon will deprecate this class.  Use installer for the set methods, use {@link RyaDetails} for the get methods.
- * New code must separate parameters that are set at Rya install time from that which is specific to the client.
- * Also Accumulo index tables are pushed down to the implementation and not configured in conf.
+ * A set of configuration utils to read a Hadoop {@link Configuration} object
+ * and create Cloudbase/Accumulo objects. Soon will deprecate this class. Use
+ * installer for the set methods, use {@link RyaDetails} for the get methods.
+ * New code must separate parameters that are set at Rya install time from that
+ * which is specific to the client. Also Accumulo index tables are pushed down
+ * to the implementation and not configured in conf.
  */
 public class ConfigUtils {
     private static final Logger logger = Logger.getLogger(ConfigUtils.class);
 
-    public static final String CLOUDBASE_TBL_PREFIX = "sc.cloudbase.tableprefix";
-    public static final String CLOUDBASE_AUTHS = "sc.cloudbase.authorizations";
-    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";
+    /**
+     * @Deprecated use {@link RdfCloudTripleStoreConfiguration#CONF_TBL_PREFIX}
+     *             instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_TBL_PREFIX = RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX;
+    /**
+     * @Deprecated use {@link AccumuloRdfConfiguration#CLOUDBASE_INSTANCE}
+     *             instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_INSTANCE = AccumuloRdfConfiguration.CLOUDBASE_INSTANCE;
+    /**
+     * @Deprecated use {@link AccumuloRdfConfiguration#CLOUDBASE_ZOOKEEPERS}
+     *             instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_ZOOKEEPERS = AccumuloRdfConfiguration.CLOUDBASE_ZOOKEEPERS;
+    /**
+     * @Deprecated use {@link AccumuloRdfConfiguration#CLOUDBASE_USER} instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_USER = AccumuloRdfConfiguration.CLOUDBASE_USER;
+    /**
+     * @Deprecated use {@link AccumuloRdfConfiguration#CLOUDBASE_PASSWORD}
+     *             instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_PASSWORD = AccumuloRdfConfiguration.CLOUDBASE_PASSWORD;
+    /**
+     * @Deprecated use {@link RdfCloudTripleStoreConfiguration#CONF_QUERY_AUTH}
+     *             instead
+     */
+    @Deprecated
+    public static final String CLOUDBASE_AUTHS = RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH;
 
     public static final String CLOUDBASE_WRITER_MAX_WRITE_THREADS = "sc.cloudbase.writer.maxwritethreads";
     public static final String CLOUDBASE_WRITER_MAX_LATENCY = "sc.cloudbase.writer.maxlatency";
@@ -95,15 +127,10 @@ public class ConfigUtils {
     public static final String USE_PCJ_UPDATER_INDEX = "sc.use.updater";
 
     public static final String FLUO_APP_NAME = "rya.indexing.pcj.fluo.fluoAppName";
-    public static final String USE_PCJ_FLUO_UPDATER = "rya.indexing.pcj.updater.fluo";
     public static final String PCJ_STORAGE_TYPE = "rya.indexing.pcj.storageType";
     public static final String PCJ_UPDATER_TYPE = "rya.indexing.pcj.updaterType";
 
-
-    public static final String USE_INDEXING_SAIL = "sc.use.indexing.sail";
-    public static final String USE_EXTERNAL_SAIL = "sc.use.external.sail";
-
-    public static final String USE_MOCK_INSTANCE = ".useMockInstance";
+    public static final String USE_MOCK_INSTANCE = AccumuloRdfConfiguration.USE_MOCK_INSTANCE;
 
     public static final String NUM_PARTITIONS = "sc.cloudbase.numPartitions";
 
@@ -125,12 +152,13 @@ public class ConfigUtils {
 
     public static final String USE_MONGO = "sc.useMongo";
 
-    public static boolean isDisplayQueryPlan(final Configuration conf){
+    public static boolean isDisplayQueryPlan(final Configuration conf) {
         return conf.getBoolean(DISPLAY_QUERY_PLAN, false);
     }
 
     /**
-     * get a value from the configuration file and throw an exception if the value does not exist.
+     * get a value from the configuration file and throw an exception if the
+     * value does not exist.
      *
      * @param conf
      * @param key
@@ -150,8 +178,8 @@ public class ConfigUtils {
      * @throws AccumuloSecurityException
      * @throws TableExistsException
      */
-    public static boolean createTableIfNotExists(final Configuration conf, final String tablename) throws AccumuloException, AccumuloSecurityException,
-            TableExistsException {
+    public static boolean createTableIfNotExists(final Configuration conf, final String tablename)
+            throws AccumuloException, AccumuloSecurityException, TableExistsException {
         final TableOperations tops = getConnector(conf).tableOperations();
         if (!tops.exists(tablename)) {
             logger.info("Creating table: " + tablename);
@@ -162,17 +190,21 @@ public class ConfigUtils {
     }
 
     /**
-     * Lookup the table name prefix in the conf and throw an error if it is null.
-     * Future,  get table prefix from RyaDetails -- the Rya instance name
-     *  -- also getting info from the RyaDetails should happen within RyaSailFactory and not ConfigUtils.
-     * @param conf  Rya configuration map where it extracts the prefix (instance name)
-     * @return  index table prefix corresponding to this Rya instance
+     * Lookup the table name prefix in the conf and throw an error if it is
+     * null. Future, get table prefix from RyaDetails -- the Rya instance name
+     * -- also getting info from the RyaDetails should happen within
+     * RyaSailFactory and not ConfigUtils.
+     * 
+     * @param conf
+     *            Rya configuration map where it extracts the prefix (instance
+     *            name)
+     * @return index table prefix corresponding to this Rya instance
      */
     public static String getTablePrefix(final Configuration conf) {
         final String tablePrefix;
         tablePrefix = conf.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
-        requireNonNull(tablePrefix, "Configuration key: " + RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX
-                + " not set.  Cannot generate table name.");
+        requireNonNull(tablePrefix,
+                "Configuration key: " + RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX + " not set.  Cannot generate table name.");
         return tablePrefix;
     }
 
@@ -187,10 +219,13 @@ public class ConfigUtils {
     public static Set<URI> getGeoPredicates(final Configuration conf) {
         return getPredicates(conf, GEO_PREDICATES_LIST);
     }
+
     /**
      * Used for indexing statements about date & time instances and intervals.
+     * 
      * @param conf
-     * @return Set of predicate URI's whose objects should be date time literals.
+     * @return Set of predicate URI's whose objects should be date time
+     *         literals.
      */
     public static Set<URI> getTemporalPredicates(final Configuration conf) {
         return getPredicates(conf, TEMPORAL_PREDICATES_LIST);
@@ -210,8 +245,8 @@ public class ConfigUtils {
         return ReflectionUtils.newInstance(c, conf);
     }
 
-    public static BatchWriter createDefaultBatchWriter(final String tablename, final Configuration conf) throws TableNotFoundException,
-            AccumuloException, AccumuloSecurityException {
+    public static BatchWriter createDefaultBatchWriter(final String tablename, final Configuration conf)
+            throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
         final Long DEFAULT_MAX_MEMORY = getWriterMaxMemory(conf);
         final Long DEFAULT_MAX_LATENCY = getWriterMaxLatency(conf);
         final Integer DEFAULT_MAX_WRITE_THREADS = getWriterMaxWriteThreads(conf);
@@ -219,7 +254,8 @@ public class ConfigUtils {
         return connector.createBatchWriter(tablename, DEFAULT_MAX_MEMORY, DEFAULT_MAX_LATENCY, DEFAULT_MAX_WRITE_THREADS);
     }
 
-    public static MultiTableBatchWriter createMultitableBatchWriter(final Configuration conf) throws AccumuloException, AccumuloSecurityException {
+    public static MultiTableBatchWriter createMultitableBatchWriter(final Configuration conf)
+            throws AccumuloException, AccumuloSecurityException {
         final Long DEFAULT_MAX_MEMORY = getWriterMaxMemory(conf);
         final Long DEFAULT_MAX_LATENCY = getWriterMaxLatency(conf);
         final Integer DEFAULT_MAX_WRITE_THREADS = getWriterMaxWriteThreads(conf);
@@ -227,26 +263,26 @@ public class ConfigUtils {
         return connector.createMultiTableBatchWriter(DEFAULT_MAX_MEMORY, DEFAULT_MAX_LATENCY, DEFAULT_MAX_WRITE_THREADS);
     }
 
-    public static Scanner createScanner(final String tablename, final Configuration conf) throws AccumuloException, AccumuloSecurityException,
-            TableNotFoundException {
+    public static Scanner createScanner(final String tablename, final Configuration conf)
+            throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
         final Connector connector = ConfigUtils.getConnector(conf);
         final Authorizations auths = ConfigUtils.getAuthorizations(conf);
         return connector.createScanner(tablename, auths);
 
     }
 
-	public static BatchScanner createBatchScanner(final String tablename, final Configuration conf) throws AccumuloException, AccumuloSecurityException,
-			TableNotFoundException {
-		final Connector connector = ConfigUtils.getConnector(conf);
-		final Authorizations auths = ConfigUtils.getAuthorizations(conf);
-		Integer numThreads = null;
-		if (conf instanceof RdfCloudTripleStoreConfiguration) {
-			numThreads = ((RdfCloudTripleStoreConfiguration) conf).getNumThreads();
-		} else {
-			numThreads = conf.getInt(RdfCloudTripleStoreConfiguration.CONF_NUM_THREADS, 2);
-		}
-		return connector.createBatchScanner(tablename, auths, numThreads);
-	}
+    public static BatchScanner createBatchScanner(final String tablename, final Configuration conf)
+            throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+        final Connector connector = ConfigUtils.getConnector(conf);
+        final Authorizations auths = ConfigUtils.getAuthorizations(conf);
+        Integer numThreads = null;
+        if (conf instanceof RdfCloudTripleStoreConfiguration) {
+            numThreads = ((RdfCloudTripleStoreConfiguration) conf).getNumThreads();
+        } else {
+            numThreads = conf.getInt(RdfCloudTripleStoreConfiguration.CONF_NUM_THREADS, 2);
+        }
+        return connector.createBatchScanner(tablename, auths, numThreads);
+    }
 
     public static int getWriterMaxWriteThreads(final Configuration conf) {
         return conf.getInt(CLOUDBASE_WRITER_MAX_WRITE_THREADS, WRITER_MAX_WRITE_THREADS);
@@ -273,7 +309,7 @@ public class ConfigUtils {
     }
 
     public static Authorizations getAuthorizations(final Configuration conf) {
-        final String authString = conf.get(CLOUDBASE_AUTHS, "");
+        final String authString = conf.get(RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, "");
         if (authString.isEmpty()) {
             return new Authorizations();
         }
@@ -325,7 +361,7 @@ public class ConfigUtils {
         return conf.getInt(FREETEXT_TERM_NUM_PARTITIONS, getNumPartitions(conf));
     }
 
-     public static boolean getUseFreeText(final Configuration conf) {
+    public static boolean getUseFreeText(final Configuration conf) {
         return conf.getBoolean(USE_FREETEXT, false);
     }
 
@@ -349,22 +385,19 @@ public class ConfigUtils {
         return conf.getBoolean(USE_PCJ_UPDATER_INDEX, false);
     }
 
-
     /**
-     * @return The name of the Fluo Application this instance of RYA is
-     *   using to incrementally update PCJs.
+     * @return The name of the Fluo Application this instance of RYA is using to
+     *         incrementally update PCJs.
      */
-    //TODO delete this eventually and use Details table
+    // TODO delete this eventually and use Details table
     public Optional<String> getFluoAppName(final Configuration conf) {
         return Optional.fromNullable(conf.get(FLUO_APP_NAME));
     }
 
-
     public static boolean getUseMongo(final Configuration conf) {
         return conf.getBoolean(USE_MONGO, false);
     }
 
-
     public static void setIndexers(final RdfCloudTripleStoreConfiguration conf) {
 
         final List<String> indexList = Lists.newArrayList();
@@ -373,25 +406,28 @@ public class ConfigUtils {
         boolean useFilterIndex = false;
 
         if (ConfigUtils.getUseMongo(conf)) {
-             if (getUseFreeText(conf)) {
+            if (getUseFreeText(conf)) {
                 indexList.add(MongoFreeTextIndexer.class.getName());
                 useFilterIndex = true;
             }
-
             if (getUseEntity(conf)) {
                 indexList.add(MongoEntityIndexer.class.getName());
                 optimizers.add(EntityIndexOptimizer.class.getName());
             }
-        } else {
 
-        	 if (getUsePCJ(conf) || getUseOptimalPCJ(conf)) {
-                 conf.setPcjOptimizer(PCJOptimizer.class);
-             }
+            if (getUseTemporal(conf)) {
+                indexList.add(MongoTemporalIndexer.class.getName());
+                useFilterIndex = true;
+            }
+        } else {
 
-             if(getUsePcjUpdaterIndex(conf)) {
-             	indexList.add(PrecomputedJoinIndexer.class.getName());
-             }
+            if (getUsePCJ(conf) || getUseOptimalPCJ(conf)) {
+                conf.setPcjOptimizer(PCJOptimizer.class);
+            }
 
+            if (getUsePcjUpdaterIndex(conf)) {
+                indexList.add(PrecomputedJoinIndexer.class.getName());
+            }
 
             if (getUseFreeText(conf)) {
                 indexList.add(AccumuloFreeTextIndexer.class.getName());
@@ -413,15 +449,13 @@ public class ConfigUtils {
             optimizers.add(FilterFunctionOptimizer.class.getName());
         }
 
-        if(conf.getUseStatementMetadata()) {
+        if (conf.getUseStatementMetadata()) {
             optimizers.add(StatementMetadataOptimizer.class.getName());
         }
 
-        conf.setStrings(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, indexList.toArray(new String[]{}));
-        conf.setStrings(RdfCloudTripleStoreConfiguration.CONF_OPTIMIZERS, optimizers.toArray(new String[]{}));
+        conf.setStrings(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, indexList.toArray(new String[] {}));
+        conf.setStrings(RdfCloudTripleStoreConfiguration.CONF_OPTIMIZERS, optimizers.toArray(new String[] {}));
 
     }
 
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/indexing/external/PrecomputedJoinIndexerConfig.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/external/PrecomputedJoinIndexerConfig.java b/extras/indexing/src/main/java/org/apache/rya/indexing/external/PrecomputedJoinIndexerConfig.java
index be1f17f..d73e180 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/external/PrecomputedJoinIndexerConfig.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/external/PrecomputedJoinIndexerConfig.java
@@ -65,7 +65,6 @@ public class PrecomputedJoinIndexerConfig {
 
     // Indicates which implementation of PrecomputedJoinUpdater to use.
     public static final String PCJ_UPDATER_TYPE = "rya.indexing.pcj.updaterType";
-    public static final String USE_PCJ_FLUO_UPDATER = ConfigUtils.USE_PCJ_FLUO_UPDATER;
 
     // The configuration object that is provided to Secondary Indexing implementations.
     private final Configuration config;
@@ -110,7 +109,7 @@ public class PrecomputedJoinIndexerConfig {
 
 
     public boolean getUseFluoUpdater() {
-    	return config.getBoolean(USE_PCJ_FLUO_UPDATER, false);
+    	return config.getBoolean(ConfigUtils.USE_PCJ_UPDATER_INDEX, false);
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java b/extras/indexing/src/main/java/org/apache/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java
index 3b01d56..ee8c295 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java
@@ -19,6 +19,8 @@
 package org.apache.rya.indexing.external.fluo;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 
 import org.apache.hadoop.conf.Configuration;
@@ -40,7 +42,7 @@ public final class FluoPcjUpdaterConfig {
     public static final String ACCUMULO_PASSWORD = ConfigUtils.CLOUDBASE_PASSWORD;
 
     // Values that define the visibilities associated with statement that are inserted by the fluo updater.
-    public static final String STATEMENT_VISIBILITY = ConfigUtils.CLOUDBASE_AUTHS;
+    public static final String STATEMENT_VISIBILITY = RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH;
 
     // The configuration object that is provided to Secondary Indexing implementations.
     private final Configuration config;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoIndexingConfiguration.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoIndexingConfiguration.java b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoIndexingConfiguration.java
new file mode 100644
index 0000000..450c20b
--- /dev/null
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoIndexingConfiguration.java
@@ -0,0 +1,418 @@
+/*
+ * 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.indexing.mongodb;
+
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.entity.EntityCentricIndex;
+import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
+import org.apache.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
+import org.apache.rya.mongodb.AbstractMongoDBRdfConfigurationBuilder;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoDBRdfConfigurationBuilder;
+import org.openrdf.sail.Sail;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class is an extension of the MongoDBRdfConfiguration object used to to
+ * create a {@link Sail} connection to an Mongo backed instance of Rya. This
+ * configuration object is designed to create Mongo Rya Sail connections where
+ * one or more of the Mongo Rya Indexes are enabled. These indexes include the
+ * {@link MongoFreeTextIndexer} and {@link MongoTemporalIndexer}
+ *
+ */
+public class MongoIndexingConfiguration extends MongoDBRdfConfiguration {
+
+    private MongoIndexingConfiguration() {
+    };
+
+    private MongoIndexingConfiguration(Configuration conf) {
+        super(conf);
+    }
+
+    public static MongoDBIndexingConfigBuilder builder() {
+        return new MongoDBIndexingConfigBuilder();
+    }
+
+    /**
+     * Creates a MongoIndexingConfiguration 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.
+     * <li>"use.freetext" - Use Mongo Freetext indexer for query and ingest.
+     * False by default.
+     * <li>"use.temporal" - Use Mongo Temporal indexer for query and ingest.
+     * False by default.
+     * <li>"use.entity" - Use Mongo Entity indexer for query and ingest. False
+     * by default.
+     * <li>"freetext.predicates" - Freetext predicates used for ingest. Specify
+     * as comma delimited Strings with no spaces between. Empty by default.
+     * <li>"temporal.predicates" - Temporal predicates used for ingest. Specify
+     * as comma delimited Strings with no spaces between. Empty by default.
+     * </ul>
+     * <br>
+     * 
+     * @param props
+     *            - Properties file containing Mongo specific configuration
+     *            parameters
+     * @return MongoIndexingConfiguration with properties set
+     */
+    public static MongoIndexingConfiguration fromProperties(Properties props) {
+        return MongoDBIndexingConfigBuilder.fromProperties(props);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link EntityCentricIndex} for ingest and at
+     * query time. The default value is false, and if useEntity is set to true
+     * and the EntityIndex does not exist, then useEntity will default to false.
+     * 
+     * @param useEntity
+     *            - use entity indexing
+     */
+    public void setUseEntity(boolean useEntity) {
+        setBoolean(ConfigUtils.USE_ENTITY, useEntity);
+    }
+
+    /**
+     * @return boolean indicating whether the {@link EntityCentricIndex} is enabled
+     */
+    public boolean getUseEntity() {
+        return getBoolean(ConfigUtils.USE_ENTITY, false);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link MongoTemporalIndexer} for ingest and
+     * at query time. The default value is false, and if useTemporal is set to
+     * true and the TemporalIndex does not exist, then useTemporal will default
+     * to false.
+     * 
+     * @param useTemporal
+     *            - use temporal indexing
+     */
+    public void setUseTemporal(boolean useTemporal) {
+        setBoolean(ConfigUtils.USE_TEMPORAL, useTemporal);
+    }
+
+    /**
+     * @return boolean indicating whether {@link MongoTemporalIndexer} is enabled
+     */
+    public boolean getUseTemporal() {
+        return getBoolean(ConfigUtils.USE_TEMPORAL, false);
+    }
+
+    /**
+     * @return boolean indicating whether {@link MongoFreeTextIndexer} is enabled
+     */
+    public boolean getUseFreetext() {
+        return getBoolean(ConfigUtils.USE_FREETEXT, false);
+    }
+
+    /**
+     * 
+     * Specify whether to use use {@link MongoFreeTextIndexer} for ingest and
+     * at query time. The default value is false, and if useFreeText is set to
+     * true and the FreeTextIndex does not exist, then useFreeText will default
+     * to false.
+     * 
+     * @param useFreeText
+     *            - use freetext indexing
+     */
+    public void setUseFreetext(boolean useFreetext) {
+        setBoolean(ConfigUtils.USE_FREETEXT, useFreetext);
+    }
+
+    /**
+     * Sets the predicates that the {@link MongoFreeTextIndexer} uses for indexing
+     * @param predicates - predicate URI used for freetext indexing
+     */
+    public void setMongoFreeTextPredicates(String[] predicates) {
+        Preconditions.checkNotNull(predicates, "Freetext predicates cannot be null.");
+        setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, predicates);
+    }
+
+    /**
+     * @return Array of predicate URI Strings used for freetext indexing
+     */
+    public String[] getMongoFreeTextPredicates() {
+        return getStrings(ConfigUtils.FREETEXT_PREDICATES_LIST);
+    }
+
+    /**
+     * Sets the predicates that the {@link MongoTemporalIndexer} uses for indexing
+     * @param predicates - predicate URI used for temporal indexing
+     */
+    public void setMongoTemporalPredicates(String[] predicates) {
+        Preconditions.checkNotNull(predicates, "Freetext predicates cannot be null.");
+        setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, predicates);
+    }
+
+    /**
+     * Gets the predicates that the {@link MongoTemporalIndexer} uses for indexing
+     * @return Array of predicate URI Strings used for temporal indexing
+     */
+    public String[] getMongoTemporalPredicates() {
+        return getStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST);
+    }
+
+    /**
+     * Concrete extension of {@link AbstractMongoDBRdfConfigurationBuilder} that
+     * adds setter methods to configure Mongo Rya indexers in addition the core
+     * Mongo Rya configuration. This builder should be used instead of
+     * {@link MongoDBRdfConfigurationBuilder} to configure a query client to use
+     * one or more Mongo Indexers.
+     *
+     */
+    public static class MongoDBIndexingConfigBuilder
+            extends AbstractMongoDBRdfConfigurationBuilder<MongoDBIndexingConfigBuilder, MongoIndexingConfiguration> {
+
+        private boolean useFreetext = false;
+        private boolean useTemporal = false;
+        private boolean useEntity = false;
+        private String[] freetextPredicates;
+        private String[] temporalPredicates;
+
+        private static final String USE_FREETEXT = "use.freetext";
+        private static final String USE_TEMPORAL = "use.temporal";
+        private static final String USE_ENTITY = "use.entity";
+        private static final String TEMPORAL_PREDICATES = "temporal.predicates";
+        private static final String FREETEXT_PREDICATES = "freetext.predicates";
+
+        /**
+         * Creates a MongoIndexingConfiguration 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.
+         * <li>"use.freetext" - Use Mongo Freetext indexer for query and ingest.
+         * False by default.
+         * <li>"use.temporal" - Use Mongo Temporal indexer for query and ingest.
+         * False by default.
+         * <li>"use.entity" - Use Mongo Entity indexer for query and ingest.
+         * False by default.
+         * <li>"freetext.predicates" - Freetext predicates used for ingest.
+         * Specify as comma delimited Strings with no spaces between. Empty by
+         * default.
+         * <li>"temporal.predicates" - Temporal predicates used for ingest.
+         * Specify as comma delimited Strings with no spaces between. Empty by
+         * default.
+         * </ul>
+         * <br>
+         * 
+         * @param props
+         *            - Properties file containing Mongo specific configuration
+         *            parameters
+         * @return MongoIndexingConfiguration with properties set
+         */
+        public static MongoIndexingConfiguration fromProperties(Properties props) {
+            try {
+                MongoDBIndexingConfigBuilder builder = new MongoDBIndexingConfigBuilder() //
+                        .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")))//
+                        .setUseMongoFreetextIndex(getBoolean(props.getProperty(USE_FREETEXT, "false")))//
+                        .setUseMongoTemporalIndex(getBoolean(props.getProperty(USE_TEMPORAL, "false")))//
+                        .setUseMongoEntityIndex(getBoolean(props.getProperty(USE_ENTITY, "false")))//
+                        .setMongoFreeTextPredicates(props.getProperty(FREETEXT_PREDICATES))//
+                        .setMongoTemporalPredicates(props.getProperty(TEMPORAL_PREDICATES));
+
+                return builder.build();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * 
+         * Specify whether to use use {@link MongoFreeTextIndexer} for ingest
+         * and at query time. The default value is false, and if useFreeText is
+         * set to true and the FreeTextIndex does not exist, then useFreeText
+         * will default to false.
+         * 
+         * @param useFreeText
+         *            - use freetext indexing
+         * @return MongoIndexingConfigBuilder for chaining method invocations
+         */
+        public MongoDBIndexingConfigBuilder setUseMongoFreetextIndex(boolean useFreeText) {
+            this.useFreetext = useFreeText;
+            return this;
+        }
+
+        /**
+         * 
+         * Specify whether to use use {@link MongoTemporalIndexer} for ingest
+         * and at query time. The default value is false, and if useTemporal is
+         * set to true and the TemporalIndex does not exist, then useTemporal
+         * will default to false.
+         * 
+         * @param useTemporal
+         *            - use temporal indexing
+         * @return MongoIndexingConfigBuilder for chaining method invocations
+         */
+        public MongoDBIndexingConfigBuilder setUseMongoTemporalIndex(boolean useTemporal) {
+            this.useTemporal = useTemporal;
+            return this;
+        }
+
+        /**
+         * 
+         * Specify whether to use the MongoEntityIndexer for ingest and at query
+         * time. The default value is false, and if useEntity is set to true and
+         * the EntityIndex does not exist, then useEntity will default to false.
+         * 
+         * @param useEntity
+         *            - use entity indexing
+         * @return MongoIndexingConfigBuilder for chaining method invocations
+         */
+        public MongoDBIndexingConfigBuilder setUseMongoEntityIndex(boolean useEntity) {
+            this.useEntity = useEntity;
+            return this;
+        }
+
+        /**
+         * 
+         * @param predicates
+         *            - String of comma delimited predicates used by the
+         *            FreetextIndexer to determine which triples to index
+         * @return MongoIndexingConfigBuilder for chaining method invocations
+         */
+        public MongoDBIndexingConfigBuilder setMongoFreeTextPredicates(String... predicates) {
+            this.freetextPredicates = predicates;
+            return this;
+        }
+
+        /**
+         * 
+         * @param predicates
+         *            - String of comma delimited predicates used by the
+         *            TemporalIndexer to determine which triples to index
+         * @return MongoIndexingConfigBuilder for chaining method invocations
+         */
+        public MongoDBIndexingConfigBuilder setMongoTemporalPredicates(String... predicates) {
+            this.temporalPredicates = predicates;
+            return this;
+        }
+
+        /**
+         * @return {@link MongoIndexingConfiguration} with specified parameters set
+         */
+        public MongoIndexingConfiguration build() {
+            MongoIndexingConfiguration conf = getConf(super.build());
+            return conf;
+        }
+
+        /**
+         * Assigns builder values to appropriate parameters within the {@link Configuration} object.
+         * 
+         * @param conf - Configuration object
+         * @return - Configuration object with parameters set
+         */
+        private MongoIndexingConfiguration getConf(MongoIndexingConfiguration conf) {
+
+            if (useFreetext) {
+                conf.setUseFreetext(useFreetext);
+                if (freetextPredicates != null) {
+                    conf.setMongoFreeTextPredicates(freetextPredicates);
+                }
+            }
+            if (useTemporal) {
+                conf.setUseTemporal(useTemporal);
+                if (temporalPredicates != null) {
+                    conf.setMongoTemporalPredicates(temporalPredicates);
+                }
+            }
+
+            conf.setUseEntity(useEntity);
+
+            return conf;
+        }
+
+        @Override
+        protected MongoDBIndexingConfigBuilder confBuilder() {
+            return this;
+        }
+
+        @Override
+        protected MongoIndexingConfiguration createConf() {
+            return new MongoIndexingConfiguration();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/FluoITBase.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/FluoITBase.java b/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/FluoITBase.java
index d378c77..a650bad 100644
--- a/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/FluoITBase.java
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/FluoITBase.java
@@ -278,7 +278,6 @@ public abstract class FluoITBase {
        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, instanceName);
        conf.set(ConfigUtils.CLOUDBASE_ZOOKEEPERS, zookeepers);
        conf.set(ConfigUtils.USE_PCJ, "true");
-       conf.set(ConfigUtils.USE_PCJ_FLUO_UPDATER, "true");
        conf.set(ConfigUtils.FLUO_APP_NAME, appName);
        conf.set(ConfigUtils.PCJ_STORAGE_TYPE, PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType.ACCUMULO.toString());
        conf.set(ConfigUtils.PCJ_UPDATER_TYPE, PrecomputedJoinIndexerConfig.PrecomputedJoinUpdaterType.FLUO.toString());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/test/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfigurationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfigurationTest.java
new file mode 100644
index 0000000..fcef816
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/accumulo/AccumuloIndexingConfigurationTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.indexing.accumulo;
+
+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.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.rya.api.domain.RyaURI;
+import org.junit.Test;
+
+public class AccumuloIndexingConfigurationTest {
+
+    @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;
+
+        AccumuloIndexingConfiguration conf = AccumuloIndexingConfiguration.builder()
+                .setAuths(auth)
+                .setVisibilities(visibility)
+                .setRyaPrefix(prefix)
+                .setUseInference(useInference)
+                .setUseCompositeCardinality(useComposite)
+                .setDisplayQueryPlan(displayPlan)
+                .setAccumuloInstance(instance)
+                .setAccumuloPassword(password)
+                .setAccumuloUser(user)
+                .setAccumuloZooKeepers(zookeeper)
+                .setUseMockAccumulo(useMock)
+                .setUseAccumuloPrefixHashing(usePrefixHash)
+                .setAccumuloFreeTextPredicates("http://pred1", "http://pred2")
+                .setAccumuloTemporalPredicates("http://pred3", "http://pred4")
+                .setUseAccumuloTemporalIndex(true)
+                .setUseAccumuloEntityIndex(true)
+                .setUseAccumuloFreetextIndex(true)
+                .setPcjUpdaterFluoAppName("fluo")
+                .setUseOptimalPcj(true)
+                .setPcjTables("table1", "table2").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);
+        assertTrue(
+                Arrays.equals(conf.getAccumuloFreeTextPredicates(), new String[] { "http://pred1", "http://pred2" }));
+        assertTrue(
+                Arrays.equals(conf.getAccumuloTemporalPredicates(), new String[] { "http://pred3", "http://pred4" }));
+        assertEquals(conf.getPcjTables(), Arrays.asList("table1", "table2"));
+        assertEquals(conf.getUsePCJ(), false);
+        assertEquals(conf.getUseOptimalPCJ(), true);
+        assertEquals(conf.getUseEntity(), true);
+        assertEquals(conf.getUseFreetext(), true);
+        assertEquals(conf.getUseTemporal(), true);
+        assertEquals(conf.getUsePCJUpdater(), true);
+        assertEquals(conf.getFluoAppUpdaterName(), "fluo");
+
+    }
+
+    @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;
+        boolean useMetadata = true;
+        Set<RyaURI> metaProperties = new HashSet<>(Arrays.asList(new RyaURI("urn:123"), new RyaURI("urn:456"))); 
+        
+
+        Properties props = new Properties();
+        props.load(new FileInputStream("src/test/resources/accumulo_rya_indexing.properties"));
+
+        AccumuloIndexingConfiguration conf = AccumuloIndexingConfiguration.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);
+        assertTrue(
+                Arrays.equals(conf.getAccumuloFreeTextPredicates(), new String[] { "http://pred1", "http://pred2" }));
+        assertTrue(
+                Arrays.equals(conf.getAccumuloTemporalPredicates(), new String[] { "http://pred3", "http://pred4" }));
+        assertEquals(conf.getPcjTables(), Arrays.asList("table1", "table2"));
+        assertEquals(conf.getUsePCJ(), false);
+        assertEquals(conf.getUseOptimalPCJ(), true);
+        assertEquals(conf.getUseEntity(), true);
+        assertEquals(conf.getUseFreetext(), true);
+        assertEquals(conf.getUseTemporal(), true);
+        assertEquals(conf.getUsePCJUpdater(), true);
+        assertEquals(conf.getFluoAppUpdaterName(), "fluo");
+        assertEquals(conf.getUseStatementMetadata(), useMetadata);
+        assertEquals(conf.getStatementMetadataProperties(), metaProperties);
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
new file mode 100644
index 0000000..cbd18b2
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.indexing.mongo;
+
+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.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.junit.Test;
+
+public class MongoIndexingConfigurationTest {
+
+    @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;
+
+        MongoIndexingConfiguration conf = MongoIndexingConfiguration.builder()
+                .setVisibilities(visibility)
+                .setUseInference(useInference)
+                .setDisplayQueryPlan(displayPlan)
+                .setUseMockMongo(useMock)
+                .setMongoCollectionPrefix(prefix)
+                .setMongoDBName("dbname")
+                .setMongoHost("host")
+                .setMongoPort("1000")
+                .setAuths(auth)
+                .setMongoUser(user)
+                .setMongoPassword(password)
+                .setUseMongoEntityIndex(true)
+                .setUseMongoFreetextIndex(true)
+                .setUseMongoTemporalIndex(true)
+                .setMongoFreeTextPredicates("http://pred1", "http://pred2")
+                .setMongoTemporalPredicates("http://pred3", "http://pred4")
+                .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);
+        assertTrue(
+                Arrays.equals(conf.getMongoFreeTextPredicates(), new String[] { "http://pred1", "http://pred2" }));
+        assertTrue(
+                Arrays.equals(conf.getMongoTemporalPredicates(), new String[] { "http://pred3", "http://pred4" }));
+
+    }
+
+    @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/mongo_rya_indexing.properties"));
+
+        MongoIndexingConfiguration conf = MongoIndexingConfiguration.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);
+        assertTrue(
+                Arrays.equals(conf.getMongoFreeTextPredicates(), new String[] { "http://pred1", "http://pred2" }));
+        assertTrue(
+                Arrays.equals(conf.getMongoTemporalPredicates(), new String[] { "http://pred3", "http://pred4" }));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/test/resources/accumulo_rya_indexing.properties
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/accumulo_rya_indexing.properties b/extras/indexing/src/test/resources/accumulo_rya_indexing.properties
new file mode 100644
index 0000000..9b4dd3d
--- /dev/null
+++ b/extras/indexing/src/test/resources/accumulo_rya_indexing.properties
@@ -0,0 +1,40 @@
+ # Licensed to the Apache Software Foundation (ASF) under one
+ # or more contributor license agreements.  See the NOTICE file
+ # distributed with this work for additional information
+ # regarding copyright ownership.  The ASF licenses this file
+ # to you under the Apache License, Version 2.0 (the
+ # "License"); you may not use this file except in compliance
+ # with the License.  You may obtain a copy of the License at
+ #
+ #   http://www.apache.org/licenses/LICENSE-2.0
+ #
+ # Unless required by applicable law or agreed to in writing,
+ # software distributed under the License is distributed on an
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ # KIND, either express or implied.  See the License for the
+ # specific language governing permissions and limitations
+ # under the License
+
+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_
+use.freetext true
+use.entity true
+use.temporal true
+use.optimal.pcj true
+freetext.predicates http://pred1,http://pred2
+temporal.predicates http://pred3,http://pred4
+pcj.tables table1,table2
+fluo.app.name fluo
+use.metadata true
+metadata.properties urn:123,urn:456
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexing/src/test/resources/mongo_rya_indexing.properties
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/mongo_rya_indexing.properties b/extras/indexing/src/test/resources/mongo_rya_indexing.properties
new file mode 100644
index 0000000..a1f7e5e
--- /dev/null
+++ b/extras/indexing/src/test/resources/mongo_rya_indexing.properties
@@ -0,0 +1,33 @@
+ # Licensed to the Apache Software Foundation (ASF) under one
+ # or more contributor license agreements.  See the NOTICE file
+ # distributed with this work for additional information
+ # regarding copyright ownership.  The ASF licenses this file
+ # to you under the Apache License, Version 2.0 (the
+ # "License"); you may not use this file except in compliance
+ # with the License.  You may obtain a copy of the License at
+ #
+ #   http://www.apache.org/licenses/LICENSE-2.0
+ #
+ # Unless required by applicable law or agreed to in writing,
+ # software distributed under the License is distributed on an
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ # KIND, either express or implied.  See the License for the
+ # specific language governing permissions and limitations
+ # under the License.
+ 
+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_
+use.freetext true
+use.temporal true
+use.entity true
+freetext.predicates http://pred1,http://pred2
+temporal.predicates http://pred3,http://pred4

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/be9ea9a3/extras/indexingExample/src/main/java/EntityDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/EntityDirectExample.java b/extras/indexingExample/src/main/java/EntityDirectExample.java
index a37e8b0..5599dcd 100644
--- a/extras/indexingExample/src/main/java/EntityDirectExample.java
+++ b/extras/indexingExample/src/main/java/EntityDirectExample.java
@@ -43,6 +43,7 @@ import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.indexing.accumulo.AccumuloIndexingConfiguration;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.sail.config.RyaSailFactory;
 
@@ -237,18 +238,15 @@ public class EntityDirectExample {
 
     private static Configuration getConf() {
 
-        final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
-
-        conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, USE_MOCK_INSTANCE);
-        conf.set(ConfigUtils.USE_ENTITY, "true");
-        conf.setTablePrefix(RYA_TABLE_PREFIX);
-        conf.set(ConfigUtils.CLOUDBASE_USER, "root");
-        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, "");
-        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, INSTANCE);
-        conf.setInt(ConfigUtils.NUM_PARTITIONS, 3);
-        conf.set(ConfigUtils.CLOUDBASE_AUTHS, AUTHS);
-
-        return conf;
+    	return AccumuloIndexingConfiguration.builder()
+    			.setUseMockAccumulo(USE_MOCK_INSTANCE)
+    			.setAuths(AUTHS)
+    			.setAccumuloUser("root")
+    			.setAccumuloPassword("")
+    			.setAccumuloInstance(INSTANCE)
+    			.setRyaPrefix(RYA_TABLE_PREFIX)
+    			.setUseAccumuloEntityIndex(true)
+    			.build();
     }