You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by mi...@apache.org on 2016/10/12 20:28:45 UTC

[5/6] incubator-rya git commit: RYA-177 adding optional geoindexing profile

RYA-177 adding optional geoindexing profile


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

Branch: refs/heads/master
Commit: 7727b1651b734d4b06d518032897f0c0dc4f88d5
Parents: 5913670
Author: pujav65 <pu...@gmail.com>
Authored: Fri Oct 7 11:26:43 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Fri Oct 7 11:26:43 2016 -0400

----------------------------------------------------------------------
 extras/indexing/pom.xml                         |  27 +-
 .../api/client/accumulo/AccumuloInstall.java    |   3 +-
 .../rya/indexing/FilterFunctionOptimizer.java   |  21 +-
 .../java/mvm/rya/indexing/GeoConstants.java     |  46 ++
 .../main/java/mvm/rya/indexing/GeoIndexer.java  | 185 -------
 .../rya/indexing/IndexingFunctionRegistry.java  |   2 -
 .../mvm/rya/indexing/accumulo/ConfigUtils.java  |  29 +-
 .../rya/indexing/accumulo/geo/GeoConstants.java |  46 --
 .../accumulo/geo/GeoMesaGeoIndexer.java         | 517 ------------------
 .../indexing/accumulo/geo/GeoParseUtils.java    | 123 -----
 .../rya/indexing/accumulo/geo/GeoTupleSet.java  | 362 -------------
 .../mongodb/geo/GeoMongoDBStorageStrategy.java  | 143 -----
 .../indexing/mongodb/geo/MongoGeoIndexer.java   | 112 ----
 .../indexing/mongodb/geo/MongoGeoTupleSet.java  | 361 -------------
 .../mvm/rya/sail/config/RyaSailFactory.java     |   2 +-
 .../indexing/accumulo/geo/GeoIndexerSfTest.java | 514 ------------------
 .../indexing/accumulo/geo/GeoIndexerTest.java   | 400 --------------
 .../indexing/mongo/MongoGeoIndexerSfTest.java   | 305 -----------
 .../rya/indexing/mongo/MongoGeoIndexerTest.java | 396 --------------
 .../src/main/java/MongoRyaDirectExample.java    | 114 ++--
 .../src/main/java/RyaDirectExample.java         | 489 +++++++++--------
 extras/pom.xml                                  |   9 +
 extras/rya.geoindexing/pom.xml                  | 224 ++++++++
 .../GeoEnabledFilterFunctionOptimizer.java      | 353 +++++++++++++
 .../main/java/mvm/rya/indexing/GeoIndexer.java  | 185 +++++++
 .../mvm/rya/indexing/GeoRyaSailFactory.java     | 146 ++++++
 .../mvm/rya/indexing/OptionalConfigUtils.java   | 141 +++++
 .../accumulo/geo/GeoMesaGeoIndexer.java         | 518 +++++++++++++++++++
 .../indexing/accumulo/geo/GeoParseUtils.java    | 119 +++++
 .../rya/indexing/accumulo/geo/GeoTupleSet.java  | 363 +++++++++++++
 .../mongodb/geo/GeoMongoDBStorageStrategy.java  | 143 +++++
 .../indexing/mongodb/geo/MongoGeoIndexer.java   | 112 ++++
 .../indexing/mongodb/geo/MongoGeoTupleSet.java  | 360 +++++++++++++
 .../indexing/accumulo/geo/GeoIndexerSfTest.java | 514 ++++++++++++++++++
 .../indexing/accumulo/geo/GeoIndexerTest.java   | 401 ++++++++++++++
 .../indexing/mongo/MongoGeoIndexerSfTest.java   | 304 +++++++++++
 .../rya/indexing/mongo/MongoGeoIndexerTest.java | 396 ++++++++++++++
 extras/rya.prospector/pom.xml                   |   1 +
 .../rya/accumulo/mr/AbstractAccumuloMRTool.java |   1 -
 .../mvm/rya/accumulo/mr/RyaOutputFormat.java    |  37 +-
 .../rya/accumulo/mr/RyaOutputFormatTest.java    |  37 --
 41 files changed, 4644 insertions(+), 3917 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/extras/indexing/pom.xml b/extras/indexing/pom.xml
index fb3042f..8e21879 100644
--- a/extras/indexing/pom.xml
+++ b/extras/indexing/pom.xml
@@ -25,15 +25,7 @@
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <geotools.version>14.3</geotools.version>
-    </properties>
-    <repositories>
-        <repository>
-            <id>osgeo</id>
-            <name>Open Source Geospatial Foundation Repository</name>
-            <url>http://download.osgeo.org/webdav/geotools/</url>
-        </repository>
-    </repositories>
+     </properties>
 
 	<dependencies>
 
@@ -82,11 +74,6 @@
 			<artifactId>commons-codec</artifactId>
 		</dependency>
 
-		<!-- Geo Indexing -->
-		<dependency>
-			<groupId>org.locationtech.geomesa</groupId>
-			<artifactId>geomesa-accumulo-datastore</artifactId>
-		</dependency>
 
 		<!-- PCJ Indexing -->
 		<dependency>
@@ -119,17 +106,7 @@
             <version>${project.version}</version>
             <type>test-jar</type>
         </dependency>
-        <dependency>
-            <groupId>org.geotools.xsd</groupId>
-            <artifactId>gt-xsd-gml3</artifactId>
-            <version>${geotools.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.geotools</groupId>
-            <artifactId>gt-api</artifactId>
-            <version>${geotools.version}</version>
-        </dependency>
-        <dependency>
+         <dependency>
             <groupId>com.vividsolutions</groupId>
             <artifactId>jts</artifactId>
             <version>1.13</version>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/api/client/accumulo/AccumuloInstall.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/api/client/accumulo/AccumuloInstall.java b/extras/indexing/src/main/java/mvm/rya/api/client/accumulo/AccumuloInstall.java
index 841be01..08c1932 100644
--- a/extras/indexing/src/main/java/mvm/rya/api/client/accumulo/AccumuloInstall.java
+++ b/extras/indexing/src/main/java/mvm/rya/api/client/accumulo/AccumuloInstall.java
@@ -186,7 +186,8 @@ public class AccumuloInstall extends AccumuloCommand implements Install {
         conf.setTablePrefix( details.getRyaInstanceName() );
 
         // Enable the indexers that the instance is configured to use.
-        conf.set(ConfigUtils.USE_GEO, "" + details.getGeoIndexDetails().isEnabled() );
+        // TODO fix me, not sure why the install command is here.
+//        conf.set(ConfigUtils.USE_GEO, "" + details.getGeoIndexDetails().isEnabled() );
         conf.set(ConfigUtils.USE_FREETEXT, "" + details.getFreeTextIndexDetails().isEnabled() );
         conf.set(ConfigUtils.USE_TEMPORAL, "" + details.getTemporalIndexDetails().isEnabled() );
         conf.set(ConfigUtils.USE_ENTITY, "" + details.getEntityCentricIndexDetails().isEnabled());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/FilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/FilterFunctionOptimizer.java b/extras/indexing/src/main/java/mvm/rya/indexing/FilterFunctionOptimizer.java
index 6c0cd9a..d148b74 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/FilterFunctionOptimizer.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/FilterFunctionOptimizer.java
@@ -34,7 +34,6 @@ import org.apache.commons.lang.Validate;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
-import org.geotools.feature.SchemaException;
 import org.openrdf.model.Resource;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -58,28 +57,21 @@ import org.openrdf.query.algebra.evaluation.QueryOptimizer;
 import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
 
 import com.google.common.collect.Lists;
-import com.mongodb.MongoClient;
 
 import mvm.rya.accumulo.AccumuloRdfConfiguration;
 import mvm.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
 import mvm.rya.indexing.accumulo.ConfigUtils;
 import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
 import mvm.rya.indexing.accumulo.freetext.FreeTextTupleSet;
-import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
-import mvm.rya.indexing.accumulo.geo.GeoTupleSet;
 import mvm.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
-import mvm.rya.indexing.mongodb.AbstractMongoIndexer;
 import mvm.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import mvm.rya.indexing.mongodb.geo.MongoGeoIndexer;
 import mvm.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
-import mvm.rya.mongodb.MongoConnectorFactory;
 
 public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     private static final Logger LOG = Logger.getLogger(FilterFunctionOptimizer.class);
     private final ValueFactory valueFactory = new ValueFactoryImpl();
 
     private Configuration conf;
-    private GeoIndexer geoIndexer;
     private FreeTextIndexer freeTextIndexer;
     private TemporalIndexer temporalIndexer;
     private boolean init = false;
@@ -88,7 +80,7 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     }
 
     public FilterFunctionOptimizer(final AccumuloRdfConfiguration conf) throws AccumuloException, AccumuloSecurityException,
-    TableNotFoundException, IOException, SchemaException, TableExistsException, NumberFormatException, UnknownHostException {
+    TableNotFoundException, IOException, TableExistsException, NumberFormatException, UnknownHostException {
         this.conf = conf;
         init();
     }
@@ -106,16 +98,12 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     private synchronized void init() {
         if (!init) {
             if (ConfigUtils.getUseMongo(conf)) {
-                    geoIndexer = new MongoGeoIndexer();
-                    geoIndexer.setConf(conf);
                     freeTextIndexer = new MongoFreeTextIndexer();
                     freeTextIndexer.setConf(conf);
                     temporalIndexer = new MongoTemporalIndexer();
                     temporalIndexer.setConf(conf);
             } else {
-                geoIndexer = new GeoMesaGeoIndexer();
-                geoIndexer.setConf(conf);
-                freeTextIndexer = new AccumuloFreeTextIndexer();
+                 freeTextIndexer = new AccumuloFreeTextIndexer();
                 freeTextIndexer.setConf(conf);
                 temporalIndexer = new AccumuloTemporalIndexer();
                 temporalIndexer.setConf(conf);
@@ -309,11 +297,6 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
         public List<TupleExpr> createReplacement(final TupleExpr org) {
             final List<TupleExpr> indexTuples = Lists.newArrayList();
             switch (type) {
-            case GEO:
-                for (final IndexingExpr indx : indxExpr) {
-                    indexTuples.add(new GeoTupleSet(indx, geoIndexer));
-                }
-                break;
             case FREETEXT:
                 for (final IndexingExpr indx : indxExpr) {
                     indexTuples.add(new FreeTextTupleSet(indx, freeTextIndexer));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/GeoConstants.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/GeoConstants.java b/extras/indexing/src/main/java/mvm/rya/indexing/GeoConstants.java
new file mode 100644
index 0000000..a692edd
--- /dev/null
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/GeoConstants.java
@@ -0,0 +1,46 @@
+package mvm.rya.indexing;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.URIImpl;
+
+/**
+ * A set of URIs used in GeoSPARQL
+ */
+public class GeoConstants {
+    public static final String NS_GEO = "http://www.opengis.net/ont/geosparql#";
+    public static final String NS_GEOF = "http://www.opengis.net/def/function/geosparql/";
+
+    public static final URI XMLSCHEMA_OGC_WKT = new URIImpl(NS_GEO + "wktLiteral");
+    public static final URI GEO_AS_WKT = new URIImpl(NS_GEO + "asWKT");
+
+    public static final URI XMLSCHEMA_OGC_GML = new URIImpl(NS_GEO + "gmlLiteral");
+    public static final URI GEO_AS_GML = new URIImpl(NS_GEO + "asGML");
+
+    public static final URI GEO_SF_EQUALS = new URIImpl(NS_GEOF + "sfEquals");
+    public static final URI GEO_SF_DISJOINT = new URIImpl(NS_GEOF + "sfDisjoint");
+    public static final URI GEO_SF_INTERSECTS = new URIImpl(NS_GEOF + "sfIntersects");
+    public static final URI GEO_SF_TOUCHES = new URIImpl(NS_GEOF + "sfTouches");
+    public static final URI GEO_SF_CROSSES = new URIImpl(NS_GEOF + "sfCrosses");
+    public static final URI GEO_SF_WITHIN = new URIImpl(NS_GEOF + "sfWithin");
+    public static final URI GEO_SF_CONTAINS = new URIImpl(NS_GEOF + "sfContains");
+    public static final URI GEO_SF_OVERLAPS = new URIImpl(NS_GEOF + "sfOverlaps");
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/GeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/GeoIndexer.java b/extras/indexing/src/main/java/mvm/rya/indexing/GeoIndexer.java
deleted file mode 100644
index 40dfeec..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/GeoIndexer.java
+++ /dev/null
@@ -1,185 +0,0 @@
-package mvm.rya.indexing;
-
-import org.openrdf.model.Statement;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.vividsolutions.jts.geom.Geometry;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-
-import info.aduna.iteration.CloseableIteration;
-import mvm.rya.api.persist.index.RyaSecondaryIndexer;
-
-/**
- * A repository to store, index, and retrieve {@link Statement}s based on geospatial features.
- */
-public interface GeoIndexer extends RyaSecondaryIndexer {
-	/**
-	 * Returns statements that contain a geometry that is equal to the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>
-	 * "Two geometries are topologically equal if their interiors intersect and no part of the interior or boundary of one geometry intersects the exterior of the other"
-	 * <li>"A is equal to B if A is within B and A contains B"
-	 * </ul>
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryEquals(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that is disjoint to the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>"A and B are disjoint if they have no point in common. They form a set of disconnected geometries."
-	 * <li>"A and B are disjoint if A does not intersect B"
-	 * </ul>
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that Intersects the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>"a intersects b: geometries a and b have at least one point in common."
-	 * <li>"not Disjoint"
-	 * </ul>
-	 *
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryIntersects(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that Touches the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>"a touches b, they have at least one boundary point in common, but no interior points."
-	 * </ul>
-	 *
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryTouches(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that crosses the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>
-	 * "a crosses b, they have some but not all interior points in common (and the dimension of the intersection is less than that of at least one of them)."
-	 * </ul>
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryCrosses(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that is Within the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>"a is within b, a lies in the interior of b"
-	 * <li>Same as: "Contains(b,a)"
-	 * </ul>
-	 *
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryWithin(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that Contains the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>b is within a. Geometry b lies in the interior of a. Another definition:
-	 * "a 'contains' b iff no points of b lie in the exterior of a, and at least one point of the interior of b lies in the interior of a"
-	 * <li>Same: Within(b,a)
-	 * </ul>
-	 *
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryContains(Geometry query, StatementConstraints contraints);
-
-	/**
-	 * Returns statements that contain a geometry that Overlaps the queried {@link Geometry} and meet the {@link StatementConstraints}.
-	 *
-	 * <p>
-	 * From Wikipedia (http://en.wikipedia.org/wiki/DE-9IM):
-	 * <ul>
-	 * <li>a crosses b, they have some but not all interior points in common (and the dimension of the intersection is less than that of at
-	 * least one of them).
-	 * </ul>
-	 *
-	 *
-	 * @param query
-	 *            the queried geometry
-	 * @param contraints
-	 *            the {@link StatementConstraints}
-	 * @return
-	 */
-	public abstract CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(Geometry query, StatementConstraints contraints);
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/IndexingFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/IndexingFunctionRegistry.java b/extras/indexing/src/main/java/mvm/rya/indexing/IndexingFunctionRegistry.java
index e96b8a3..2f2c486 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/IndexingFunctionRegistry.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/IndexingFunctionRegistry.java
@@ -24,8 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import mvm.rya.indexing.accumulo.geo.GeoConstants;
-
 import org.openrdf.model.URI;
 import org.openrdf.model.impl.URIImpl;
 import org.openrdf.query.algebra.ValueConstant;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java
index 01c76db..7c608de 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java
@@ -58,11 +58,9 @@ import mvm.rya.indexing.accumulo.entity.EntityOptimizer;
 import mvm.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
 import mvm.rya.indexing.accumulo.freetext.LuceneTokenizer;
 import mvm.rya.indexing.accumulo.freetext.Tokenizer;
-import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
 import mvm.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
 import mvm.rya.indexing.external.PrecomputedJoinIndexer;
 import mvm.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import mvm.rya.indexing.mongodb.geo.MongoGeoIndexer;
 import mvm.rya.indexing.pcj.matching.PCJOptimizer;
 
 /**
@@ -87,9 +85,6 @@ public class ConfigUtils {
 
     public static final String FREE_TEXT_QUERY_TERM_LIMIT = "sc.freetext.querytermlimit";
 
-    public static final String GEO_NUM_PARTITIONS = "sc.geo.numPartitions";
-
-    public static final String USE_GEO = "sc.use_geo";
     public static final String USE_FREETEXT = "sc.use_freetext";
     public static final String USE_TEMPORAL = "sc.use_temporal";
     public static final String USE_ENTITY = "sc.use_entity";
@@ -199,7 +194,7 @@ public class ConfigUtils {
         return getPredicates(conf, TEMPORAL_PREDICATES_LIST);
     }
 
-    private static Set<URI> getPredicates(final Configuration conf, final String confName) {
+    protected static Set<URI> getPredicates(final Configuration conf, final String confName) {
         final String[] validPredicateStrings = conf.getStrings(confName, new String[] {});
         final Set<URI> predicates = new HashSet<URI>();
         for (final String prediateString : validPredicateStrings) {
@@ -316,7 +311,7 @@ public class ConfigUtils {
         return conf.getBoolean(USE_MOCK_INSTANCE, false);
     }
 
-    private static int getNumPartitions(final Configuration conf) {
+    protected static int getNumPartitions(final Configuration conf) {
         return conf.getInt(NUM_PARTITIONS, 25);
     }
 
@@ -328,15 +323,7 @@ public class ConfigUtils {
         return conf.getInt(FREETEXT_TERM_NUM_PARTITIONS, getNumPartitions(conf));
     }
 
-    public static int getGeoNumPartitions(final Configuration conf) {
-        return conf.getInt(GEO_NUM_PARTITIONS, getNumPartitions(conf));
-    }
-
-    public static boolean getUseGeo(final Configuration conf) {
-        return conf.getBoolean(USE_GEO, false);
-    }
-
-    public static boolean getUseFreeText(final Configuration conf) {
+     public static boolean getUseFreeText(final Configuration conf) {
         return conf.getBoolean(USE_FREETEXT, false);
     }
 
@@ -384,11 +371,7 @@ public class ConfigUtils {
         boolean useFilterIndex = false;
 
         if (ConfigUtils.getUseMongo(conf)) {
-            if (getUseGeo(conf)) {
-                indexList.add(MongoGeoIndexer.class.getName());
-                useFilterIndex = true;
-            }
-            if (getUseFreeText(conf)) {
+             if (getUseFreeText(conf)) {
                 indexList.add(MongoFreeTextIndexer.class.getName());
                 useFilterIndex = true;
             }
@@ -402,10 +385,6 @@ public class ConfigUtils {
              	indexList.add(PrecomputedJoinIndexer.class.getName());
              }
 
-            if (getUseGeo(conf)) {
-                indexList.add(GeoMesaGeoIndexer.class.getName());
-                useFilterIndex = true;
-            }
 
             if (getUseFreeText(conf)) {
                 indexList.add(AccumuloFreeTextIndexer.class.getName());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoConstants.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoConstants.java b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoConstants.java
deleted file mode 100644
index 8f563e0..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoConstants.java
+++ /dev/null
@@ -1,46 +0,0 @@
-package mvm.rya.indexing.accumulo.geo;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import org.openrdf.model.URI;
-import org.openrdf.model.impl.URIImpl;
-
-/**
- * A set of URIs used in GeoSPARQL
- */
-public class GeoConstants {
-    public static final String NS_GEO = "http://www.opengis.net/ont/geosparql#";
-    public static final String NS_GEOF = "http://www.opengis.net/def/function/geosparql/";
-
-    public static final URI XMLSCHEMA_OGC_WKT = new URIImpl(NS_GEO + "wktLiteral");
-    public static final URI GEO_AS_WKT = new URIImpl(NS_GEO + "asWKT");
-
-    public static final URI XMLSCHEMA_OGC_GML = new URIImpl(NS_GEO + "gmlLiteral");
-    public static final URI GEO_AS_GML = new URIImpl(NS_GEO + "asGML");
-
-    public static final URI GEO_SF_EQUALS = new URIImpl(NS_GEOF + "sfEquals");
-    public static final URI GEO_SF_DISJOINT = new URIImpl(NS_GEOF + "sfDisjoint");
-    public static final URI GEO_SF_INTERSECTS = new URIImpl(NS_GEOF + "sfIntersects");
-    public static final URI GEO_SF_TOUCHES = new URIImpl(NS_GEOF + "sfTouches");
-    public static final URI GEO_SF_CROSSES = new URIImpl(NS_GEOF + "sfCrosses");
-    public static final URI GEO_SF_WITHIN = new URIImpl(NS_GEOF + "sfWithin");
-    public static final URI GEO_SF_CONTAINS = new URIImpl(NS_GEOF + "sfContains");
-    public static final URI GEO_SF_OVERLAPS = new URIImpl(NS_GEOF + "sfOverlaps");
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
deleted file mode 100644
index 7bfe6b7..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
+++ /dev/null
@@ -1,517 +0,0 @@
-package mvm.rya.indexing.accumulo.geo;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Logger;
-import org.geotools.data.DataStore;
-import org.geotools.data.DataStoreFinder;
-import org.geotools.data.DataUtilities;
-import org.geotools.data.FeatureSource;
-import org.geotools.data.FeatureStore;
-import org.geotools.data.Query;
-import org.geotools.factory.CommonFactoryFinder;
-import org.geotools.factory.Hints;
-import org.geotools.feature.DefaultFeatureCollection;
-import org.geotools.feature.FeatureIterator;
-import org.geotools.feature.SchemaException;
-import org.geotools.feature.simple.SimpleFeatureBuilder;
-import org.geotools.filter.text.cql2.CQLException;
-import org.geotools.filter.text.ecql.ECQL;
-import org.locationtech.geomesa.accumulo.data.AccumuloDataStore;
-import org.locationtech.geomesa.accumulo.index.Constants;
-import org.locationtech.geomesa.utils.geotools.SimpleFeatureTypes;
-import org.opengis.feature.simple.SimpleFeature;
-import org.opengis.feature.simple.SimpleFeatureType;
-import org.opengis.filter.Filter;
-import org.opengis.filter.FilterFactory;
-import org.opengis.filter.identity.Identifier;
-import org.openrdf.model.Literal;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.io.ParseException;
-import com.vividsolutions.jts.io.WKTReader;
-
-import info.aduna.iteration.CloseableIteration;
-import mvm.rya.accumulo.experimental.AbstractAccumuloIndexer;
-import mvm.rya.api.RdfCloudTripleStoreConfiguration;
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RyaToRdfConversions;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.Md5Hash;
-import mvm.rya.indexing.StatementConstraints;
-import mvm.rya.indexing.StatementSerializer;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-
-/**
- * A {@link GeoIndexer} wrapper around a GeoMesa {@link AccumuloDataStore}. This class configures and connects to the Datastore, creates the
- * RDF Feature Type, and interacts with the Datastore.
- * <p>
- * Specifically, this class creates a RDF Feature type and stores each RDF Statement as a RDF Feature in the datastore. Each feature
- * contains the standard set of GeoMesa attributes (Geometry, Start Date, and End Date). The GeoMesaGeoIndexer populates the Geometry
- * attribute by parsing the Well-Known Text contained in the RDF Statement\u2019s object literal value.
- * <p>
- * The RDF Feature contains four additional attributes for each component of the RDF Statement. These attributes are:
- * <p>
- * <table border="1">
- * <tr>
- * <th>Name</th>
- * <th>Symbol</th>
- * <th>Type</th>
- * </tr>
- * <tr>
- * <td>Subject Attribute</td>
- * <td>S</td>
- * <td>String</td>
- * </tr>
- * </tr>
- * <tr>
- * <td>Predicate Attribute</td>
- * <td>P</td>
- * <td>String</td>
- * </tr>
- * </tr>
- * <tr>
- * <td>Object Attribute</td>
- * <td>O</td>
- * <td>String</td>
- * </tr>
- * </tr>
- * <tr>
- * <td>Context Attribute</td>
- * <td>C</td>
- * <td>String</td>
- * </tr>
- * </table>
- */
-public class GeoMesaGeoIndexer extends AbstractAccumuloIndexer implements GeoIndexer  {
-
-    private static final String TABLE_SUFFIX = "geo";
-
-    private static final Logger logger = Logger.getLogger(GeoMesaGeoIndexer.class);
-
-    private static final String FEATURE_NAME = "RDF";
-
-    private static final String SUBJECT_ATTRIBUTE = "S";
-    private static final String PREDICATE_ATTRIBUTE = "P";
-    private static final String OBJECT_ATTRIBUTE = "O";
-    private static final String CONTEXT_ATTRIBUTE = "C";
-
-    private Set<URI> validPredicates;
-    private Configuration conf;
-    private FeatureStore<SimpleFeatureType, SimpleFeature> featureStore;
-    private FeatureSource<SimpleFeatureType, SimpleFeature> featureSource;
-    private SimpleFeatureType featureType;
-    private boolean isInit = false;
-
-    //initialization occurs in setConf because index is created using reflection
-    @Override
-    public void setConf(final Configuration conf) {
-        this.conf = conf;
-        if (!isInit) {
-            try {
-            	initInternal();
-                isInit = true;
-            } catch (final IOException e) {
-                logger.warn("Unable to initialize index.  Throwing Runtime Exception. ", e);
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-
-
-    private void initInternal() throws IOException {
-        validPredicates = ConfigUtils.getGeoPredicates(conf);
-
-        final DataStore dataStore = createDataStore(conf);
-
-        try {
-            featureType = getStatementFeatureType(dataStore);
-        } catch (final IOException e) {
-            throw new IOException(e);
-        } catch (final SchemaException e) {
-            throw new IOException(e);
-        }
-
-        featureSource = dataStore.getFeatureSource(featureType.getName());
-        if (!(featureSource instanceof FeatureStore)) {
-            throw new IllegalStateException("Could not retrieve feature store");
-        }
-        featureStore = (FeatureStore<SimpleFeatureType, SimpleFeature>) featureSource;
-    }
-
-    private static DataStore createDataStore(final Configuration conf) throws IOException {
-        // get the configuration parameters
-        final Instance instance = ConfigUtils.getInstance(conf);
-        final boolean useMock = instance instanceof MockInstance;
-        final String instanceId = instance.getInstanceName();
-        final String zookeepers = instance.getZooKeepers();
-        final String user = ConfigUtils.getUsername(conf);
-        final String password = ConfigUtils.getPassword(conf);
-        final String auths = ConfigUtils.getAuthorizations(conf).toString();
-        final String tableName = getTableName(conf);
-        final int numParitions = ConfigUtils.getGeoNumPartitions(conf);
-
-        final String featureSchemaFormat = "%~#s%" + numParitions + "#r%" + FEATURE_NAME
-                + "#cstr%0,3#gh%yyyyMMdd#d::%~#s%3,2#gh::%~#s%#id";
-        // build the map of parameters
-        final Map<String, Serializable> params = new HashMap<String, Serializable>();
-        params.put("instanceId", instanceId);
-        params.put("zookeepers", zookeepers);
-        params.put("user", user);
-        params.put("password", password);
-        params.put("auths", auths);
-        params.put("tableName", tableName);
-        params.put("indexSchemaFormat", featureSchemaFormat);
-        params.put("useMock", Boolean.toString(useMock));
-
-        // fetch the data store from the finder
-        return DataStoreFinder.getDataStore(params);
-    }
-
-    private static SimpleFeatureType getStatementFeatureType(final DataStore dataStore) throws IOException, SchemaException {
-        SimpleFeatureType featureType;
-
-        final String[] datastoreFeatures = dataStore.getTypeNames();
-        if (Arrays.asList(datastoreFeatures).contains(FEATURE_NAME)) {
-            featureType = dataStore.getSchema(FEATURE_NAME);
-        } else {
-            final String featureSchema = SUBJECT_ATTRIBUTE + ":String," //
-                    + PREDICATE_ATTRIBUTE + ":String," //
-                    + OBJECT_ATTRIBUTE + ":String," //
-                    + CONTEXT_ATTRIBUTE + ":String," //
-                    + Constants.SF_PROPERTY_GEOMETRY + ":Geometry:srid=4326";
-            featureType = SimpleFeatureTypes.createType(FEATURE_NAME, featureSchema);
-            dataStore.createSchema(featureType);
-        }
-        return featureType;
-    }
-
-    @Override
-    public void storeStatements(final Collection<RyaStatement> ryaStatements) throws IOException {
-        // create a feature collection
-        final DefaultFeatureCollection featureCollection = new DefaultFeatureCollection();
-        for (final RyaStatement ryaStatement : ryaStatements) {
-            final Statement statement = RyaToRdfConversions.convertStatement(ryaStatement);
-            // if the predicate list is empty, accept all predicates.
-            // Otherwise, make sure the predicate is on the "valid" list
-            final boolean isValidPredicate = validPredicates.isEmpty() || validPredicates.contains(statement.getPredicate());
-
-            if (isValidPredicate && (statement.getObject() instanceof Literal)) {
-                try {
-                    final SimpleFeature feature = createFeature(featureType, statement);
-                    featureCollection.add(feature);
-                } catch (final ParseException e) {
-                    logger.warn("Error getting geo from statement: " + statement.toString(), e);
-                }
-            }
-        }
-
-        // write this feature collection to the store
-        if (!featureCollection.isEmpty()) {
-            featureStore.addFeatures(featureCollection);
-        }
-    }
-
-
-    @Override
-    public void storeStatement(final RyaStatement statement) throws IOException {
-        storeStatements(Collections.singleton(statement));
-    }
-
-    private static SimpleFeature createFeature(final SimpleFeatureType featureType, final Statement statement) throws ParseException {
-        final String subject = StatementSerializer.writeSubject(statement);
-        final String predicate = StatementSerializer.writePredicate(statement);
-        final String object = StatementSerializer.writeObject(statement);
-        final String context = StatementSerializer.writeContext(statement);
-
-        // create the feature
-        final Object[] noValues = {};
-
-        // create the hash
-        final String statementId = Md5Hash.md5Base64(StatementSerializer.writeStatement(statement));
-        final SimpleFeature newFeature = SimpleFeatureBuilder.build(featureType, noValues, statementId);
-
-        // write the statement data to the fields
-        final Geometry geom = GeoParseUtils.getGeometry(statement); 
-        if(geom == null || geom.isEmpty() || !geom.isValid()) {
-            throw new ParseException("Could not create geometry for statement " + statement);
-        }
-        newFeature.setDefaultGeometry(geom);
-
-        newFeature.setAttribute(SUBJECT_ATTRIBUTE, subject);
-        newFeature.setAttribute(PREDICATE_ATTRIBUTE, predicate);
-        newFeature.setAttribute(OBJECT_ATTRIBUTE, object);
-        newFeature.setAttribute(CONTEXT_ATTRIBUTE, context);
-
-        // preserve the ID that we created for this feature
-        // (set the hint to FALSE to have GeoTools generate IDs)
-        newFeature.getUserData().put(Hints.USE_PROVIDED_FID, java.lang.Boolean.TRUE);
-
-        return newFeature;
-    }
-
-    private CloseableIteration<Statement, QueryEvaluationException> performQuery(final String type, final Geometry geometry,
-            final StatementConstraints contraints) {
-        final List<String> filterParms = new ArrayList<String>();
-
-        filterParms.add(type + "(" + Constants.SF_PROPERTY_GEOMETRY + ", " + geometry + " )");
-
-        if (contraints.hasSubject()) {
-            filterParms.add("( " + SUBJECT_ATTRIBUTE + "= '" + contraints.getSubject() + "') ");
-        }
-        if (contraints.hasContext()) {
-            filterParms.add("( " + CONTEXT_ATTRIBUTE + "= '" + contraints.getContext() + "') ");
-        }
-        if (contraints.hasPredicates()) {
-            final List<String> predicates = new ArrayList<String>();
-            for (final URI u : contraints.getPredicates()) {
-                predicates.add("( " + PREDICATE_ATTRIBUTE + "= '" + u.stringValue() + "') ");
-            }
-            filterParms.add("(" + StringUtils.join(predicates, " OR ") + ")");
-        }
-
-        final String filterString = StringUtils.join(filterParms, " AND ");
-        logger.info("Performing geomesa query : " + filterString);
-
-        return getIteratorWrapper(filterString);
-    }
-
-    private CloseableIteration<Statement, QueryEvaluationException> getIteratorWrapper(final String filterString) {
-
-        return new CloseableIteration<Statement, QueryEvaluationException>() {
-
-            private FeatureIterator<SimpleFeature> featureIterator = null;
-
-            FeatureIterator<SimpleFeature> getIterator() throws QueryEvaluationException {
-                if (featureIterator == null) {
-                    Filter cqlFilter;
-                    try {
-                        cqlFilter = ECQL.toFilter(filterString);
-                    } catch (final CQLException e) {
-                        logger.error("Error parsing query: " + filterString, e);
-                        throw new QueryEvaluationException(e);
-                    }
-
-                    final Query query = new Query(featureType.getTypeName(), cqlFilter);
-                    try {
-                        featureIterator = featureSource.getFeatures(query).features();
-                    } catch (final IOException e) {
-                        logger.error("Error performing query: " + filterString, e);
-                        throw new QueryEvaluationException(e);
-                    }
-
-                }
-                return featureIterator;
-            }
-
-            @Override
-            public boolean hasNext() throws QueryEvaluationException {
-                return getIterator().hasNext();
-            }
-
-            @Override
-            public Statement next() throws QueryEvaluationException {
-                final SimpleFeature feature = getIterator().next();
-                final String subjectString = feature.getAttribute(SUBJECT_ATTRIBUTE).toString();
-                final String predicateString = feature.getAttribute(PREDICATE_ATTRIBUTE).toString();
-                final String objectString = feature.getAttribute(OBJECT_ATTRIBUTE).toString();
-                final String contextString = feature.getAttribute(CONTEXT_ATTRIBUTE).toString();
-                final Statement statement = StatementSerializer.readStatement(subjectString, predicateString, objectString, contextString);
-                return statement;
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException("Remove not implemented");
-            }
-
-            @Override
-            public void close() throws QueryEvaluationException {
-                getIterator().close();
-            }
-        };
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryEquals(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("EQUALS", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("DISJOINT", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("INTERSECTS", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryTouches(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("TOUCHES", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("CROSSES", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryWithin(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("WITHIN", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryContains(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("CONTAINS", query, contraints);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(final Geometry query, final StatementConstraints contraints) {
-        return performQuery("OVERLAPS", query, contraints);
-    }
-
-    @Override
-    public Set<URI> getIndexablePredicates() {
-        return validPredicates;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        // TODO cache and flush features instead of writing them one at a time
-    }
-
-    @Override
-    public void close() throws IOException {
-        flush();
-    }
-
-
-    @Override
-    public String getTableName() {
-            return getTableName(conf);
-    }
-
-    /**
-     * Get the Accumulo table that will be used by this index.  
-     * @param conf
-     * @return table name guaranteed to be used by instances of this index
-     */
-    public static String getTableName(Configuration conf) {
-        return ConfigUtils.getTablePrefix(conf)  + TABLE_SUFFIX;
-    }
-
-    private void deleteStatements(final Collection<RyaStatement> ryaStatements) throws IOException {
-        // create a feature collection
-        final DefaultFeatureCollection featureCollection = new DefaultFeatureCollection();
-
-        for (final RyaStatement ryaStatement : ryaStatements) {
-            final Statement statement = RyaToRdfConversions.convertStatement(ryaStatement);
-            // if the predicate list is empty, accept all predicates.
-            // Otherwise, make sure the predicate is on the "valid" list
-            final boolean isValidPredicate = validPredicates.isEmpty() || validPredicates.contains(statement.getPredicate());
-
-            if (isValidPredicate && (statement.getObject() instanceof Literal)) {
-                try {
-                    final SimpleFeature feature = createFeature(featureType, statement);
-                    featureCollection.add(feature);
-                } catch (final ParseException e) {
-                    logger.warn("Error getting geo from statement: " + statement.toString(), e);
-                }
-            }
-        }
-
-        // remove this feature collection from the store
-        if (!featureCollection.isEmpty()) {
-            final Set<Identifier> featureIds = new HashSet<Identifier>();
-            final FilterFactory filterFactory = CommonFactoryFinder.getFilterFactory(null);
-            final Set<String> stringIds = DataUtilities.fidSet(featureCollection);
-            for (final String id : stringIds) {
-                featureIds.add(filterFactory.featureId(id));
-            }
-            final Filter filter = filterFactory.id(featureIds);
-            featureStore.removeFeatures(filter);
-        }
-    }
-
-
-    @Override
-    public void deleteStatement(final RyaStatement statement) throws IOException {
-        deleteStatements(Collections.singleton(statement));
-    }
-
-	@Override
-	public void init() {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public void setConnector(final Connector connector) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public void destroy() {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public void purge(final RdfCloudTripleStoreConfiguration configuration) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public void dropAndDestroy() {
-		// TODO Auto-generated method stub
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoParseUtils.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoParseUtils.java b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoParseUtils.java
deleted file mode 100644
index 119f351..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoParseUtils.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package mvm.rya.indexing.accumulo.geo;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.io.StringReader;
-import java.nio.charset.StandardCharsets;
-
-import javax.xml.parsers.ParserConfigurationException;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-import org.apache.log4j.Logger;
-import org.geotools.data.collection.ListFeatureCollection;
-import org.geotools.gml3.GMLConfiguration;
-import org.geotools.xml.Parser;
-import org.openrdf.model.Literal;
-import org.openrdf.model.Statement;
-import org.xml.sax.SAXException;
-
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.io.ParseException;
-import com.vividsolutions.jts.io.WKTReader;
-
-import mvm.rya.api.domain.RyaStatement;
-
-public class GeoParseUtils {
-    static final Logger logger = Logger.getLogger(GeoParseUtils.class);
-    /** 
-     * @deprecated  Not needed since geo literals may be WKT or GML.
-     *
-     *    This method warns on a condition that must already be tested.  Replaced by
-     *    {@link #getLiteral(Statement)} and {@link #getGeometry(Statement}
-     *    and getLiteral(statement).toString()
-     *    and getLiteral(statement).getDatatype()
-     */
-    @Deprecated
-	public static String getWellKnownText(Statement statement) throws ParseException {
-	    Literal lit = getLiteral(statement);
-	    if (!GeoConstants.XMLSCHEMA_OGC_WKT.equals(lit.getDatatype())) {
-	        logger.warn("Literal is not of type " + GeoConstants.XMLSCHEMA_OGC_WKT + ": " + statement.toString());
-	    }
-	    return lit.getLabel().toString();
-	}
-
-    public static Literal getLiteral(Statement statement) throws ParseException {
-        org.openrdf.model.Value v = statement.getObject();
-        if (!(v instanceof Literal)) {
-            throw new ParseException("Statement does not contain Literal: " + statement.toString());
-        }
-        Literal lit = (Literal) v;
-        return lit;
-    }
-
-    /**
-     * Parse GML/wkt literal to Geometry
-     * 
-     * @param statement
-     * @return
-     * @throws ParseException
-     * @throws ParserConfigurationException 
-     * @throws SAXException 
-     * @throws IOException 
-     */
-    public static Geometry getGeometry(Statement statement) throws ParseException {
-        // handle GML or WKT
-        Literal lit = getLiteral(statement);
-        if (GeoConstants.XMLSCHEMA_OGC_WKT.equals(lit.getDatatype())) {
-            final String wkt = lit.getLabel().toString();
-            return (new WKTReader()).read(wkt);
-        } else if (GeoConstants.XMLSCHEMA_OGC_GML.equals(lit.getDatatype())) {
-            String gml = lit.getLabel().toString();
-            try {
-                return getGeometryGml(gml);
-            } catch (IOException | SAXException | ParserConfigurationException e) {
-                throw new ParseException(e);
-            }
-        } else {
-            throw new ParseException("Literal is unknown geo type, expecting WKT or GML: " + statement.toString());
-        }
-    }
-    /**
-     * Convert GML/XML string into a geometry that can be indexed.
-     * @param gmlString
-     * @return
-     * @throws IOException
-     * @throws SAXException
-     * @throws ParserConfigurationException
-     */
-    public static Geometry getGeometryGml(String gmlString) throws IOException, SAXException, ParserConfigurationException {
-        Reader reader = new StringReader(gmlString); 
-        GMLConfiguration gmlConfiguration = new GMLConfiguration();
-        Parser gmlParser = new Parser(gmlConfiguration);
-        //  gmlParser.setStrict(false);  // attempt at allowing deprecated elements, but no.
-        //  gmlParser.setValidating(false);
-        final Geometry geometry = (Geometry) gmlParser.parse(reader);
-        // This sometimes gets populated with the SRS/CRS: geometry.getUserData()
-        // Always returns 0 : geometry.getSRID() 
-        //TODO geometry.setUserData(some default CRS); OR geometry.setSRID(some default CRS)
-        
-        return geometry;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoTupleSet.java b/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoTupleSet.java
deleted file mode 100644
index d1468b8..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/accumulo/geo/GeoTupleSet.java
+++ /dev/null
@@ -1,362 +0,0 @@
-package mvm.rya.indexing.accumulo.geo;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Maps;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.io.ParseException;
-import com.vividsolutions.jts.io.WKTReader;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-import info.aduna.iteration.CloseableIteration;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.IndexingExpr;
-import mvm.rya.indexing.IteratorFactory;
-import mvm.rya.indexing.SearchFunction;
-import mvm.rya.indexing.StatementConstraints;
-import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
-
-//Indexing Node for geo expressions to be inserted into execution plan
-//to delegate geo portion of query to geo index
-public class GeoTupleSet extends ExternalTupleSet {
-
-    private final Configuration conf;
-    private final GeoIndexer geoIndexer;
-    private final IndexingExpr filterInfo;
-
-
-    public GeoTupleSet(final IndexingExpr filterInfo, final GeoIndexer geoIndexer) {
-        this.filterInfo = filterInfo;
-        this.geoIndexer = geoIndexer;
-        conf = geoIndexer.getConf();
-    }
-
-    @Override
-    public Set<String> getBindingNames() {
-        return filterInfo.getBindingNames();
-    }
-
-    @Override
-	public GeoTupleSet clone() {
-        return new GeoTupleSet(filterInfo, geoIndexer);
-    }
-
-    @Override
-    public double cardinality() {
-        return 0.0; // No idea how the estimate cardinality here.
-    }
-
-
-    @Override
-    public String getSignature() {
-        return "(GeoTuple Projection) " + "variables: " + Joiner.on(", ").join(getBindingNames()).replaceAll("\\s+", " ");
-    }
-
-
-
-    @Override
-    public boolean equals(final Object other) {
-        if (other == this) {
-            return true;
-        }
-        if (!(other instanceof GeoTupleSet)) {
-            return false;
-        }
-        final GeoTupleSet arg = (GeoTupleSet) other;
-        return filterInfo.equals(arg.filterInfo);
-    }
-
-    @Override
-    public int hashCode() {
-        int result = 17;
-        result = 31*result + filterInfo.hashCode();
-
-        return result;
-    }
-
-
-
-    /**
-     * Returns an iterator over the result set of the contained IndexingExpr.
-     * <p>
-     * Should be thread-safe (concurrent invocation {@link OfflineIterable} this
-     * method can be expected with some query evaluators.
-     */
-    @Override
-    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindings)
-            throws QueryEvaluationException {
-
-
-        final URI funcURI = filterInfo.getFunction();
-        final SearchFunction searchFunction = new GeoSearchFunctionFactory(conf).getSearchFunction(funcURI);
-        if(filterInfo.getArguments().length > 1) {
-            throw new IllegalArgumentException("Index functions do not support more than two arguments.");
-        }
-
-        final String queryText = filterInfo.getArguments()[0].stringValue();
-
-        return IteratorFactory.getIterator(filterInfo.getSpConstraint(), bindings, queryText, searchFunction);
-    }
-
-
-
-    //returns appropriate search function for a given URI
-    //search functions used in GeoMesaGeoIndexer to access index
-    public class GeoSearchFunctionFactory {
-
-        Configuration conf;
-
-        private final Map<URI, SearchFunction> SEARCH_FUNCTION_MAP = Maps.newHashMap();
-
-        public GeoSearchFunctionFactory(final Configuration conf) {
-            this.conf = conf;
-        }
-
-
-        /**
-         * Get a {@link GeoSearchFunction} for a given URI.
-         *
-         * @param searchFunction
-         * @return
-         */
-        public SearchFunction getSearchFunction(final URI searchFunction) {
-
-            SearchFunction geoFunc = null;
-
-            try {
-                geoFunc = getSearchFunctionInternal(searchFunction);
-            } catch (final QueryEvaluationException e) {
-                e.printStackTrace();
-            }
-
-            return geoFunc;
-        }
-
-        private SearchFunction getSearchFunctionInternal(final URI searchFunction) throws QueryEvaluationException {
-            final SearchFunction sf = SEARCH_FUNCTION_MAP.get(searchFunction);
-
-            if (sf != null) {
-                return sf;
-            } else {
-                throw new QueryEvaluationException("Unknown Search Function: " + searchFunction.stringValue());
-            }
-        }
-
-        private final SearchFunction GEO_EQUALS = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_EQUALS";
-            };
-        };
-
-        private final SearchFunction GEO_DISJOINT = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_DISJOINT";
-            };
-        };
-
-        private final SearchFunction GEO_INTERSECTS = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_INTERSECTS";
-            };
-        };
-
-        private final SearchFunction GEO_TOUCHES = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_TOUCHES";
-            };
-        };
-
-        private final SearchFunction GEO_CONTAINS = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_CONTAINS";
-            };
-        };
-
-        private final SearchFunction GEO_OVERLAPS = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_OVERLAPS";
-            };
-        };
-
-        private final SearchFunction GEO_CROSSES = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_CROSSES";
-            };
-        };
-
-        private final SearchFunction GEO_WITHIN = new SearchFunction() {
-
-            @Override
-            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
-                    final StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    final WKTReader reader = new WKTReader();
-                    final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (final ParseException e) {
-                    throw new QueryEvaluationException(e);
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "GEO_WITHIN";
-            };
-        };
-
-        {
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_EQUALS, GEO_EQUALS);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_DISJOINT, GEO_DISJOINT);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_INTERSECTS, GEO_INTERSECTS);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_TOUCHES, GEO_TOUCHES);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_CONTAINS, GEO_CONTAINS);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_OVERLAPS, GEO_OVERLAPS);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_CROSSES, GEO_CROSSES);
-            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_WITHIN, GEO_WITHIN);
-        }
-
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
deleted file mode 100644
index 9411330..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
+++ /dev/null
@@ -1,143 +0,0 @@
-package mvm.rya.indexing.mongodb.geo;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.log4j.Logger;
-import org.openrdf.model.Statement;
-
-import com.mongodb.BasicDBObject;
-import com.mongodb.DBCollection;
-import com.mongodb.DBObject;
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.io.ParseException;
-import com.vividsolutions.jts.io.WKTReader;
-
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.resolver.RyaToRdfConversions;
-import mvm.rya.indexing.accumulo.geo.GeoParseUtils;
-import mvm.rya.indexing.mongodb.IndexingMongoDBStorageStrategy;
-
-public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
-    private static final Logger LOG = Logger.getLogger(GeoMongoDBStorageStrategy.class);
-
-    private static final String GEO = "location";
-    public enum GeoQueryType {
-        INTERSECTS {
-            @Override
-            public String getKeyword() {
-                return "$geoIntersects";
-            }
-        }, WITHIN {
-            @Override
-            public String getKeyword() {
-                return "$geoWithin";
-            }
-        },
-        EQUALS {
-            @Override
-            public String getKeyword() {
-                return "$near";
-            }
-        };
-
-        public abstract String getKeyword();
-    }
-
-    static class GeoQuery {
-        private final GeoQueryType queryType;
-        private final Geometry geo;
-
-        public GeoQuery(final GeoQueryType queryType, final Geometry geo) {
-            this.queryType = queryType;
-            this.geo = geo;
-        }
-
-        public GeoQueryType getQueryType() {
-            return queryType;
-        }
-        public Geometry getGeo() {
-            return geo;
-        }
-    }
-
-    private final double maxDistance;
-
-    public GeoMongoDBStorageStrategy(final double maxDistance) {
-        this.maxDistance = maxDistance;
-    }
-
-    @Override
-    public void createIndices(final DBCollection coll){
-        coll.createIndex("{" + GEO + " : \"2dsphere\"" );
-    }
-
-    public DBObject getQuery(final GeoQuery queryObj) {
-        final Geometry geo = queryObj.getGeo();
-        final GeoQueryType queryType = queryObj.getQueryType();
-
-        BasicDBObject query;
-        if (queryType.equals(GeoQueryType.EQUALS)){
-            final List<double[]> points = getCorrespondingPoints(geo);
-            if (points.size() == 1){
-                final List circle = new ArrayList();
-                circle.add(points.get(0));
-                circle.add(maxDistance);
-                final BasicDBObject polygon = new BasicDBObject("$centerSphere", circle);
-                query = new BasicDBObject(GEO,  new BasicDBObject(GeoQueryType.WITHIN.getKeyword(), polygon));
-            } else {
-                query = new BasicDBObject(GEO, points);
-            }
-        } else {
-            query = new BasicDBObject(GEO, new BasicDBObject(queryType.getKeyword(), new BasicDBObject("$polygon", getCorrespondingPoints(geo))));
-        }
-
-        return query;
-    }
-
-    @Override
-    public DBObject serialize(final RyaStatement ryaStatement) {
-        // if the object is wkt, then try to index it
-        // write the statement data to the fields
-        try {
-            final Statement statement = RyaToRdfConversions.convertStatement(ryaStatement);
-            final Geometry geo = (new WKTReader()).read(GeoParseUtils.getWellKnownText(statement));
-            final BasicDBObject base = (BasicDBObject) super.serialize(ryaStatement);
-            base.append(GEO, getCorrespondingPoints(geo));
-            return base;
-        } catch(final ParseException e) {
-            LOG.error("Could not create geometry for statement " + ryaStatement, e);
-            return null;
-        }
-    }
-
-    private List<double[]> getCorrespondingPoints(final Geometry geo){
-       final List<double[]> points = new ArrayList<double[]>();
-        for (final Coordinate coord : geo.getCoordinates()){
-            points.add(new double[] {
-                coord.x, coord.y
-            });
-        }
-        return points;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7727b165/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/MongoGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/MongoGeoIndexer.java b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/MongoGeoIndexer.java
deleted file mode 100644
index 7589f03..0000000
--- a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/geo/MongoGeoIndexer.java
+++ /dev/null
@@ -1,112 +0,0 @@
-package mvm.rya.indexing.mongodb.geo;
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import static mvm.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.EQUALS;
-import static mvm.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.INTERSECTS;
-import static mvm.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.WITHIN;
-
-import org.apache.log4j.Logger;
-import org.openrdf.model.Statement;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.mongodb.DBObject;
-import com.vividsolutions.jts.geom.Geometry;
-
-import info.aduna.iteration.CloseableIteration;
-import mvm.rya.indexing.GeoIndexer;
-import mvm.rya.indexing.StatementConstraints;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-import mvm.rya.indexing.mongodb.AbstractMongoIndexer;
-import mvm.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQuery;
-import mvm.rya.mongodb.MongoDBRdfConfiguration;
-
-public class MongoGeoIndexer extends AbstractMongoIndexer<GeoMongoDBStorageStrategy> implements GeoIndexer {
-    private static final String COLLECTION_SUFFIX = "geo";
-    private static final Logger logger = Logger.getLogger(MongoGeoIndexer.class);
-
-    @Override
-	public void init() {
-        initCore();
-        predicates = ConfigUtils.getGeoPredicates(conf);
-        storageStrategy = new GeoMongoDBStorageStrategy(Double.valueOf(conf.get(MongoDBRdfConfiguration.MONGO_GEO_MAXDISTANCE, "1e-10")));
-        storageStrategy.createIndices(collection);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryEquals(
-            final Geometry query, final StatementConstraints constraints) {
-        final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(EQUALS, query));
-        return withConstraints(constraints, queryObj);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Disjoint queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(
-            final Geometry query, final StatementConstraints constraints) {
-        final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(INTERSECTS, query));
-        return withConstraints(constraints, queryObj);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryTouches(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Touches queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Crosses queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryWithin(
-            final Geometry query, final StatementConstraints constraints) {
-        final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(WITHIN, query));
-        return withConstraints(constraints, queryObj);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryContains(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Contains queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Overlaps queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public String getCollectionName() {
-        return ConfigUtils.getTablePrefix(conf)  + COLLECTION_SUFFIX;
-    }
-}
\ No newline at end of file