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 2017/08/03 04:47:24 UTC

[1/3] incubator-rya git commit: RYA-317 Added Mongo Near support Closes #186

Repository: incubator-rya
Updated Branches:
  refs/heads/master 2ca854271 -> 94911ed89


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
new file mode 100644
index 0000000..389cc28
--- /dev/null
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
@@ -0,0 +1,286 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.indexing.GeoRyaSailFactory;
+import org.apache.rya.indexing.OptionalConfigUtils;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
+import org.apache.rya.mongodb.MockMongoFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.TupleQueryResult;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.Sail;
+
+import com.mongodb.MongoClient;
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.io.ParseException;
+import com.vividsolutions.jts.io.WKTReader;
+import com.vividsolutions.jts.io.WKTWriter;
+
+public class MongoGeoIndexerFilterIT {
+    private static final GeometryFactory GF = new GeometryFactory();
+    private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
+    private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));
+    private static final Geometry CAPITAL_BUILDING = GF.createPoint(new Coordinate(38.8899, 77.0091));
+    private static final Geometry WHITE_HOUSE = GF.createPoint(new Coordinate(38.8977, 77.0365));
+
+    private MongoClient client;
+    private Sail sail;
+    private SailRepositoryConnection conn;
+
+    @Before
+    public void before() throws Exception {
+        final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
+            .setMongoCollectionPrefix("rya_")
+            .setMongoDBName("indexerTests")
+            .setUseMongoFreetextIndex(false)
+            .setUseMongoTemporalIndex(false)
+            .build();
+
+        client = MockMongoFactory.newFactory().newMongoClient();
+        indxrConf.setBoolean(OptionalConfigUtils.USE_GEO, true);
+        indxrConf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        indxrConf.setBoolean(ConfigUtils.USE_MONGO, true);
+        indxrConf.setMongoClient(client);
+
+        sail = GeoRyaSailFactory.getInstance(indxrConf);
+        conn = new SailRepository(sail).getConnection();
+        conn.begin();
+    }
+
+    @Test
+    public void nearHappyUsesTest() throws Exception {
+        populateRya();
+
+        //Only captial
+        String query =
+             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+           + "SELECT * \n" //
+           + "WHERE { \n"
+           + "  <urn:geo> geo:asWKT ?point .\n"
+           + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 0.0, 2000))"
+           + "}";
+
+        TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        final List<BindingSet> results = new ArrayList<>();
+        while (rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            results.add(bs);
+        }
+        assertEquals(1, results.size());
+        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(0)));
+
+        //all but capital
+        query =
+            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+          + "SELECT * \n" //
+          + "WHERE { \n"
+          + "  <urn:geo> geo:asWKT ?point .\n"
+          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000))"
+          + "}";
+
+        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        results.clear();
+        while (rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            results.add(bs);
+        }
+        assertEquals(3, results.size());
+        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+
+        // all of them
+        query =
+            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+          + "SELECT * \n" //
+          + "WHERE { \n"
+          + "  <urn:geo> geo:asWKT ?point .\n"
+          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 6000, 000))"
+          + "}";
+
+        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        results.clear();
+        while (rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            results.add(bs);
+        }
+        assertEquals(4, results.size());
+        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+
+        // donut, only 2
+        query =
+            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+          + "SELECT * \n" //
+          + "WHERE { \n"
+          + "  <urn:geo> geo:asWKT ?point .\n"
+          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000, 100))"
+          + "}";
+
+        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        results.clear();
+        while (rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            results.add(bs);
+        }
+        assertEquals(2, results.size());
+        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(0)));
+        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(1)));
+
+        // all of them
+        query =
+            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+          + "SELECT * \n" //
+          + "WHERE { \n"
+          + "  <urn:geo> geo:asWKT ?point .\n"
+          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral))"
+          + "}";
+        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        results.clear();
+        while (rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            results.add(bs);
+        }
+        assertEquals(4, results.size());
+        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+    }
+
+    @Test(expected = MalformedQueryException.class)
+    public void near_invalidDistance() throws Exception {
+        populateRya();
+
+        //Only captial
+        final String query =
+             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+           + "SELECT * \n" //
+           + "WHERE { \n"
+           + "  <urn:geo> geo:asWKT ?point .\n"
+                + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, distance))"
+           + "}";
+
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void near_negativeDistance() throws Exception {
+        populateRya();
+
+        //Only captial
+        final String query =
+             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+           + "SELECT * \n" //
+           + "WHERE { \n"
+           + "  <urn:geo> geo:asWKT ?point .\n"
+           + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, -100))"
+           + "}";
+
+        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        while(rez.hasNext()) {
+            rez.next();
+        }
+    }
+
+    @Test(expected = QueryEvaluationException.class)
+    public void tooManyArgumentsTest() throws Exception {
+        populateRya();
+
+        // Only captial
+        final String query =
+              "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+            + "SELECT * \n" //
+            + "WHERE { \n" + "  <urn:geo> geo:asWKT ?point .\n"
+            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 100, 1000, 10))"
+            + "}";
+
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+    }
+
+    private void populateRya() throws Exception {
+        // geo 2x2 points
+        RyaStatement stmnt = statement(WASHINGTON_MONUMENT);
+        Statement statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(LINCOLN_MEMORIAL);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(CAPITAL_BUILDING);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(WHITE_HOUSE);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+    }
+
+    private static Geometry bindingToGeo(final BindingSet bs) throws ParseException {
+        final WKTReader w = new WKTReader();
+        return w.read(bs.getValue("point").stringValue());
+    }
+
+    private static RyaStatement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("urn:geo");
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final WKTWriter w = new WKTWriter();
+        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
new file mode 100644
index 0000000..b533d42
--- /dev/null
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
@@ -0,0 +1,173 @@
+/*
+ * 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 org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.indexing.GeoRyaSailFactory;
+import org.apache.rya.indexing.OptionalConfigUtils;
+import org.apache.rya.indexing.TemporalInstant;
+import org.apache.rya.indexing.TemporalInstantRfc3339;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
+import org.apache.rya.mongodb.MockMongoFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.Update;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.Sail;
+
+import com.mongodb.MongoClient;
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.io.WKTWriter;
+
+public class MongoIndexerDeleteIT {
+    private MongoClient client;
+    private Sail sail;
+    private SailRepositoryConnection conn;
+
+    @Before
+    public void before() throws Exception {
+        final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
+            .setMongoCollectionPrefix("rya_")
+            .setMongoDBName("indexerTests")
+            .setUseMongoFreetextIndex(true)
+            .setUseMongoTemporalIndex(true)
+            .setMongoFreeTextPredicates(RDFS.LABEL.stringValue())
+            .setMongoTemporalPredicates("Property:atTime")
+            .build();
+
+        client = MockMongoFactory.newFactory().newMongoClient();
+        indxrConf.setBoolean(OptionalConfigUtils.USE_GEO, true);
+        indxrConf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        indxrConf.setBoolean(ConfigUtils.USE_MONGO, true);
+        indxrConf.setMongoClient(client);
+
+        sail = GeoRyaSailFactory.getInstance(indxrConf);
+        conn = new SailRepository(sail).getConnection();
+        conn.begin();
+    }
+
+    @Test
+    public void deleteTest() throws Exception {
+        populateRya();
+
+        //The extra 1 is from the person type defined in freetext
+        assertEquals(8, client.getDatabase("indexerTests").getCollection("rya__triples").count());
+        assertEquals(4, client.getDatabase("indexerTests").getCollection("rya_rya_geo").count());
+        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
+        assertEquals(2, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
+
+        //free text -- remove one from many
+        String delete = "DELETE DATA \n" //
+           + "{\n"
+           + "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
+           + "}";
+        Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+        update.execute();
+
+        // temporal -- remove one from one
+        delete = "DELETE DATA \n" //
+           + "{\n"
+           + "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
+           + "}";
+
+        update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+        update.execute();
+
+        //geo -- remove many from many
+        delete =
+             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+           + "DELETE \n" //
+           + "{\n"
+           + "  <urn:geo> geo:asWKT ?point \n"
+           + "}"
+           + "WHERE { \n"
+           + "  <urn:geo> geo:asWKT ?point .\n"
+           + "  FILTER(geof:sfWithin(?point, \"POLYGON((0 0, 2 0, 2 1, 0 1, 0 0))\"^^geo:wktLiteral))"
+           + "}";
+
+        update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+        update.execute();
+
+        assertEquals(2, client.getDatabase("indexerTests").getCollection("rya_rya_geo").count());
+        assertEquals(0, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
+        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
+        assertEquals(4, client.getDatabase("indexerTests").getCollection("rya__triples").count());
+    }
+
+    private void populateRya() throws Exception {
+        final ValueFactory VF = new ValueFactoryImpl();
+        // geo 2x2 points
+        final GeometryFactory GF = new GeometryFactory();
+        for (int x = 0; x <= 1; x++) {
+            for (int y = 0; y <= 1; y++) {
+                final Geometry geo = GF.createPoint(new Coordinate(x + .5, y + .5));
+                final RyaStatement stmnt = statement(geo);
+                final Statement statement = RyaToRdfConversions.convertStatement(stmnt);
+                conn.add(statement);
+            }
+        }
+
+        // freetext
+        final URI person = VF.createURI("http://example.org/ontology/Person");
+        String uuid;
+
+        uuid = "urn:people";
+        conn.add(VF.createURI(uuid), RDF.TYPE, person);
+        conn.add(VF.createURI(uuid), RDFS.LABEL, VF.createLiteral("Alice Palace Hose", VF.createURI("http://www.w3.org/2001/XMLSchema#string")));
+
+        uuid = "urn:people";
+        conn.add(VF.createURI(uuid), RDF.TYPE, person);
+        conn.add(VF.createURI(uuid), RDFS.LABEL, VF.createLiteral("Bob Snob Hose", "en"));
+
+        // temporal
+        final TemporalInstant instant = new TemporalInstantRfc3339(1, 2, 3, 4, 5, 6);
+        final URI time = VF.createURI("Property:atTime");
+        conn.add(VF.createURI("foo:time"), VF.createURI("Property:atTime"), VF.createLiteral(instant.toString()));
+    }
+
+    private static RyaStatement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("urn:geo");
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final WKTWriter w = new WKTWriter();
+        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+    }
+
+}


[2/3] incubator-rya git commit: RYA-317 Added Mongo Near support Closes #186

Posted by mi...@apache.org.
RYA-317 Added Mongo Near support Closes #186

Support for mongo added.
fixed so no longer always calls sfWithin
properly enabled Geo indexer in mongo


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

Branch: refs/heads/master
Commit: 5e26bbe28358a26bb7fc9256d653df71e3cbb5d6
Parents: 2ca8542
Author: isper3at <sm...@gmail.com>
Authored: Tue Jul 25 16:15:38 2017 -0400
Committer: Aaron Mihalik <aa...@gmail.com>
Committed: Thu Aug 3 00:07:42 2017 -0400

----------------------------------------------------------------------
 .../apache/rya/mongodb/MongoRyaTestBase.java    |   2 +-
 .../org/apache/rya/indexing/GeoConstants.java   |   5 +-
 .../rya/indexing/IndexingFunctionRegistry.java  |   1 +
 .../indexing/mongodb/AbstractMongoIndexer.java  |   6 +-
 .../org/apache/rya/indexing/GeoIndexer.java     |  25 ++
 .../accumulo/geo/GeoMesaGeoIndexer.java         |   9 +-
 .../rya/indexing/accumulo/geo/GeoTupleSet.java  | 154 +++++++-
 .../accumulo/geo/GeoWaveGeoIndexer.java         |   7 +
 .../mongo/EventDocumentConverter.java           |  72 ++--
 .../GeoTemporalMongoDBStorageStrategy.java      |  10 +-
 .../mongodb/geo/GeoMongoDBStorageStrategy.java  | 131 ++++++-
 .../indexing/mongodb/geo/MongoGeoIndexer.java   |  23 ++
 .../indexing/mongodb/geo/MongoGeoTupleSet.java  | 360 -------------------
 .../GeoTemporalMongoDBStorageStrategyTest.java  | 163 +++++----
 .../apache/rya/indexing/mongo/MongoGeoIT.java   | 153 --------
 .../indexing/mongo/MongoGeoIndexerFilterIT.java | 286 +++++++++++++++
 .../indexing/mongo/MongoIndexerDeleteIT.java    | 173 +++++++++
 17 files changed, 926 insertions(+), 654 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
index 272767a..b0a4161 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
@@ -25,11 +25,11 @@ import org.apache.rya.api.persist.RyaDAOException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
 
 import com.mongodb.MongoClient;
 
 import de.flapdoodle.embed.mongo.distribution.Version;
-import org.junit.BeforeClass;
 
 public class MongoRyaTestBase {
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/indexing/src/main/java/org/apache/rya/indexing/GeoConstants.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/GeoConstants.java b/extras/indexing/src/main/java/org/apache/rya/indexing/GeoConstants.java
index ec3df7f..2cb8217 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/GeoConstants.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/GeoConstants.java
@@ -8,9 +8,9 @@ package org.apache.rya.indexing;
  * 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
@@ -43,4 +43,5 @@ public class GeoConstants {
     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");
+    public static final URI GEO_SF_NEAR = new URIImpl(NS_GEOF + "sfNear");
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
index a21b41d..b61af4b 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
@@ -61,6 +61,7 @@ public class IndexingFunctionRegistry {
         SEARCH_FUNCTIONS.put(GeoConstants.GEO_SF_CONTAINS, FUNCTION_TYPE.GEO);
         SEARCH_FUNCTIONS.put(GeoConstants.GEO_SF_OVERLAPS, FUNCTION_TYPE.GEO);
         SEARCH_FUNCTIONS.put(GeoConstants.GEO_SF_CROSSES, FUNCTION_TYPE.GEO);
+        SEARCH_FUNCTIONS.put(GeoConstants.GEO_SF_NEAR, FUNCTION_TYPE.GEO);
 
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
index 2428e28..f5372d1 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
@@ -127,8 +127,8 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
 
     @Override
     public void deleteStatement(final RyaStatement stmt) throws IOException {
-       final DBObject obj = storageStrategy.getQuery(stmt);
-       collection.remove(obj);
+        final DBObject obj = storageStrategy.getQuery(stmt);
+        collection.remove(obj);
     }
 
     @Override
@@ -151,7 +151,7 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
                 }
             }
         } catch (final IllegalArgumentException e) {
-            LOG.error("Unable to parse the statement: " + ryaStatement.toString());
+            LOG.error("Unable to parse the statement: " + ryaStatement.toString(), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoIndexer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoIndexer.java
index d71f036..d091d32 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoIndexer.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoIndexer.java
@@ -28,6 +28,7 @@ import com.vividsolutions.jts.geom.Geometry;
 
 import info.aduna.iteration.CloseableIteration;
 import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
+import org.apache.rya.indexing.accumulo.geo.GeoTupleSet.GeoSearchFunctionFactory.NearQuery;
 
 /**
  * A repository to store, index, and retrieve {@link Statement}s based on geospatial features.
@@ -182,4 +183,28 @@ public interface GeoIndexer extends RyaSecondaryIndexer {
 	 * @return
 	 */
 	public abstract CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(Geometry query, StatementConstraints contraints);
+	
+    /**
+     * Returns statements that contain a geometry that is near the queried {@link Geometry} and meet the {@link StatementConstraints}.
+     * <p>
+     * A geometry is considered near if it within the min/max distances specified in the provided {@link NearQuery}.  This will make a disc (specify max),
+     *  a donut(specify both), or a spheroid complement disc (specify min)
+     * <p>
+     * The distances are specified in meters and must be >= 0.
+     * <p>
+     * To specify max/min distances:
+     * <ul>
+     * <li>Enter parameters in order MAX, MIN -- Donut</li>
+     * <li>Omit the MIN -- Disc</li>
+     * <li>Enter 0 for MAX, and Enter parameter for MIN -- Spheroid complement Dist</li>
+     * <li>Omit both -- Default max/min [TODO: Find these values]</li>
+     * </ul>
+     * <p>
+     * Note: This query will not fail if the min is greater than the max, it will just return no results.
+     * 
+     * @param query the queried geometry, with Optional min and max distance fields.
+     * @param contraints the {@link StatementConstraints}
+     * @return
+     */
+    public abstract CloseableIteration<Statement, QueryEvaluationException> queryNear(NearQuery query, StatementConstraints contraints);
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
index 1956355..12a84fd 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoMesaGeoIndexer.java
@@ -48,6 +48,7 @@ import org.apache.rya.indexing.OptionalConfigUtils;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.StatementSerializer;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.geo.GeoTupleSet.GeoSearchFunctionFactory.NearQuery;
 import org.geotools.data.DataStore;
 import org.geotools.data.DataStoreFinder;
 import org.geotools.data.DataUtilities;
@@ -78,7 +79,7 @@ import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.io.ParseException;
 
 import info.aduna.iteration.CloseableIteration;
-
+ 
 /**
  * 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.
@@ -410,6 +411,12 @@ public class GeoMesaGeoIndexer extends AbstractAccumuloIndexer implements GeoInd
     public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(final Geometry query, final StatementConstraints contraints) {
         return performQuery("OVERLAPS", query, contraints);
     }
+    
+    @Override
+    public CloseableIteration<Statement, QueryEvaluationException> queryNear(final NearQuery query,
+            final StatementConstraints contraints) {
+        throw new UnsupportedOperationException("Near queries are not supported in Accumulo.");
+    }
 
     @Override
     public Set<URI> getIndexablePredicates() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
index d00b849..3f4be9c 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
@@ -1,8 +1,12 @@
 package org.apache.rya.indexing.accumulo.geo;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
+import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.GeoIndexer;
@@ -13,6 +17,7 @@ import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
+import org.openrdf.model.Value;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.QueryEvaluationException;
 
@@ -43,11 +48,12 @@ import com.vividsolutions.jts.io.WKTReader;
 
 
 import info.aduna.iteration.CloseableIteration;
+import joptsimple.internal.Strings;
 
 //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 static final String NEAR_DELIM = "::";
     private final Configuration conf;
     private final GeoIndexer geoIndexer;
     private final IndexingExpr filterInfo;
@@ -114,19 +120,35 @@ public class GeoTupleSet extends ExternalTupleSet {
     public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindings)
             throws QueryEvaluationException {
 
-
         final URI funcURI = filterInfo.getFunction();
         final SearchFunction searchFunction = new GeoSearchFunctionFactory(conf, geoIndexer).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();
+        String queryText = filterInfo.getArguments()[0].stringValue();
 
-        return IteratorFactory.getIterator(filterInfo.getSpConstraint(), bindings, queryText, searchFunction);
-    }
+        if(funcURI.equals(GeoConstants.GEO_SF_NEAR)) {
+            if (filterInfo.getArguments().length > 3) {
+                throw new IllegalArgumentException("Near functions do not support more than four arguments.");
+            }
 
+            final List<String> valueList = new ArrayList<>();
+            for (final Value val : filterInfo.getArguments()) {
+                valueList.add(val.stringValue());
+            }
+            queryText = Strings.join(valueList, NEAR_DELIM);
+        } else if (filterInfo.getArguments().length > 1) {
+            throw new IllegalArgumentException("Index functions do not support more than two arguments.");
+        }
 
+        try {
+            final CloseableIteration<BindingSet, QueryEvaluationException> iterrez = IteratorFactory
+                    .getIterator(filterInfo.getSpConstraint(), bindings,
+                    queryText, searchFunction);
+            return iterrez;
+        } catch (final Exception e) {
+            System.out.println(e.getMessage());
+            throw e;
+        }
+    }
 
     //returns appropriate search function for a given URI
     //search functions used in GeoMesaGeoIndexer to access index
@@ -181,7 +203,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryEquals(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -203,7 +225,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryDisjoint(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -225,7 +247,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryIntersects(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -247,7 +269,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryTouches(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -269,7 +291,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryContains(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -291,7 +313,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryOverlaps(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -313,7 +335,7 @@ public class GeoTupleSet extends ExternalTupleSet {
                 try {
                     final WKTReader reader = new WKTReader();
                     final Geometry geometry = reader.read(queryText);
-                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                    final CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryCrosses(
                             geometry, contraints);
                     return statements;
                 } catch (final ParseException e) {
@@ -349,6 +371,104 @@ public class GeoTupleSet extends ExternalTupleSet {
             };
         };
 
+        private final SearchFunction GEO_NEAR = new SearchFunction() {
+            @Override
+            public CloseableIteration<Statement, QueryEvaluationException> performSearch(final String queryText,
+                    final StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    final String[] args = queryText.split(NEAR_DELIM);
+                    Optional<Double> maxDistanceOpt = Optional.empty();
+                    Optional<Double> minDistanceOpt = Optional.empty();
+                    final String query = args[0];
+
+                    for (int ii = 1; ii < args.length; ii++) {
+                        String numArg = args[ii];
+
+                        // remove pre-padding 0's since NumberUtils.isNumber()
+                        // will assume its octal if it starts with a 0.
+                        while (numArg.startsWith("0")) {
+                            numArg = numArg.substring(1);
+                        }
+                        // was 0
+                        if (numArg.equals("")) {
+                            // if max hasn't been set, set it to 0.
+                            // Otherwise, min is just ignored.
+                            if (!maxDistanceOpt.isPresent()) {
+                                maxDistanceOpt = Optional.of(0.0);
+                            }
+                        } else {
+                            if (!maxDistanceOpt.isPresent() && NumberUtils.isNumber(numArg)) {
+                                // no variable identifier, going by order.
+                                maxDistanceOpt = getDistanceOpt(numArg, "maxDistance");
+                            } else if (NumberUtils.isNumber(numArg)) {
+                                // no variable identifier, going by order.
+                                minDistanceOpt = getDistanceOpt(numArg, "minDistance");
+                            } else {
+                                throw new IllegalArgumentException(numArg + " is not a valid Near function argument.");
+                            }
+                        }
+                    }
+                    final WKTReader reader = new WKTReader();
+                    final Geometry geometry = reader.read(query);
+                    final NearQuery nearQuery = new NearQuery(maxDistanceOpt, minDistanceOpt, geometry);
+                    return geoIndexer.queryNear(nearQuery, contraints);
+                } catch (final ParseException e) {
+                    throw new QueryEvaluationException(e);
+                }
+            }
+
+            private Optional<Double> getDistanceOpt(final String num, final String name) {
+                try {
+                    double dist = Double.parseDouble(num);
+                    if(dist < 0) {
+                        throw new IllegalArgumentException("Value for: " + name + " must be non-negative.");
+                    }
+                    return Optional.of(Double.parseDouble(num));
+                } catch (final NumberFormatException nfe) {
+                    throw new IllegalArgumentException("Value for: " + name + " must be a number.");
+                }
+            }
+
+            @Override
+            public String toString() {
+                return "GEO_NEAR";
+            }
+        };
+
+        /**
+         *
+         */
+        public class NearQuery {
+            private final Optional<Double> maxDistanceOpt;
+            private final Optional<Double> minDistanceOpt;
+            private final Geometry geo;
+
+            /**
+             *
+             * @param maxDistance
+             * @param minDistance
+             * @param geo
+             */
+            public NearQuery(final Optional<Double> maxDistance, final Optional<Double> minDistance,
+                    final Geometry geo) {
+                maxDistanceOpt = maxDistance;
+                minDistanceOpt = minDistance;
+                this.geo = geo;
+            }
+
+            public Optional<Double> getMaxDistance() {
+                return maxDistanceOpt;
+            }
+
+            public Optional<Double> getMinDistance() {
+                return minDistanceOpt;
+            }
+
+            public Geometry getGeometry() {
+                return geo;
+            }
+        }
+
         {
             SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_EQUALS, GEO_EQUALS);
             SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_DISJOINT, GEO_DISJOINT);
@@ -358,9 +478,7 @@ public class GeoTupleSet extends ExternalTupleSet {
             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);
+            SEARCH_FUNCTION_MAP.put(GeoConstants.GEO_SF_NEAR, GEO_NEAR);
         }
-
     }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoWaveGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoWaveGeoIndexer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoWaveGeoIndexer.java
index 520ae81..45a23f9 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoWaveGeoIndexer.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoWaveGeoIndexer.java
@@ -49,6 +49,7 @@ import org.apache.rya.indexing.Md5Hash;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.StatementSerializer;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.geo.GeoTupleSet.GeoSearchFunctionFactory.NearQuery;
 import org.geotools.data.DataStore;
 import org.geotools.data.DataUtilities;
 import org.geotools.data.FeatureSource;
@@ -478,6 +479,12 @@ public class GeoWaveGeoIndexer extends AbstractAccumuloIndexer implements GeoInd
     public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(final Geometry query, final StatementConstraints contraints) {
         return performQuery("OVERLAPS", query, contraints);
     }
+    
+    @Override
+    public CloseableIteration<Statement, QueryEvaluationException> queryNear(final NearQuery query,
+            final StatementConstraints contraints) {
+        throw new UnsupportedOperationException("Near queries are not supported in Accumulo.");
+    }
 
     @Override
     public Set<URI> getIndexablePredicates() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/EventDocumentConverter.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/EventDocumentConverter.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/EventDocumentConverter.java
index a41428e..926f357 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/EventDocumentConverter.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/EventDocumentConverter.java
@@ -31,12 +31,13 @@ import org.apache.rya.indexing.entity.storage.mongo.DocumentConverter;
 import org.apache.rya.indexing.geotemporal.model.Event;
 import org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy;
 import org.bson.Document;
-import org.bson.types.BasicBSONList;
 import org.joda.time.DateTime;
 
 import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.CoordinateList;
 import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.LinearRing;
 
 public class EventDocumentConverter implements DocumentConverter<Event>{
     public static final String SUBJECT = "_id";
@@ -45,7 +46,7 @@ public class EventDocumentConverter implements DocumentConverter<Event>{
     public static final String INTERVAL_END = "end";
     public static final String INSTANT = "instant";
 
-    private final GeoMongoDBStorageStrategy geoAdapter = new GeoMongoDBStorageStrategy(0);
+    private final GeoMongoDBStorageStrategy geoAdapter = new GeoMongoDBStorageStrategy(0.0);
 
     @Override
     public Document toDocument(final Event event) {
@@ -55,16 +56,11 @@ public class EventDocumentConverter implements DocumentConverter<Event>{
         doc.append(SUBJECT, event.getSubject().getData());
 
         if(event.getGeometry().isPresent()) {
-            final BasicBSONList points = new BasicBSONList();
-            for(final double[] point : geoAdapter.getCorrespondingPoints(event.getGeometry().get())) {
-                final BasicBSONList pointDoc = new BasicBSONList();
-                for(final double p : point) {
-                    pointDoc.add(p);
-                }
-                points.add(pointDoc);
+            if (event.getGeometry().get().getNumPoints() > 1) {
+                doc.append(GEO_KEY, geoAdapter.getCorrespondingPoints(event.getGeometry().get()));
+            } else {
+                doc.append(GEO_KEY, geoAdapter.getDBPoint(event.getGeometry().get()));
             }
-
-            doc.append(GEO_KEY, points);
         }
         if(event.isInstant()) {
             if(event.getInstant().isPresent()) {
@@ -104,21 +100,50 @@ public class EventDocumentConverter implements DocumentConverter<Event>{
             .setSubject(new RyaURI(subject));
 
         if(document.containsKey(GEO_KEY)) {
-            final List<List<Double>> pointsList = (List<List<Double>>) document.get(GEO_KEY);
-            final Coordinate[] coords = new Coordinate[pointsList.size()];
-
-            int ii = 0;
-            for(final List<Double> point : pointsList) {
-                coords[ii] = new Coordinate(point.get(0), point.get(1));
-                ii++;
-            }
-
+            final Document geoObj = (Document) document.get(GEO_KEY);
             final GeometryFactory geoFact = new GeometryFactory();
+            final String typeString = (String) geoObj.get("type");
+            final CoordinateList coords = new CoordinateList();
             final Geometry geo;
-            if(coords.length == 1) {
-                geo = geoFact.createPoint(coords[0]);
+            if (typeString.equals("Point")) {
+                final List<Double> point = (List<Double>) geoObj.get("coordinates");
+                final Coordinate coord = new Coordinate(point.get(0), point.get(1));
+                geo = geoFact.createPoint(coord);
+            } else if (typeString.equals("LineString")) {
+                final List<List<Double>> pointsList = (List<List<Double>>) geoObj.get("coordinates");
+                for (final List<Double> point : pointsList) {
+                    coords.add(new Coordinate(point.get(0), point.get(1)));
+                }
+                geo = geoFact.createLineString(coords.toCoordinateArray());
             } else {
-                geo = geoFact.createPolygon(coords);
+                final List<List<List<Double>>> pointsList = (List<List<List<Double>>>) geoObj.get("coordinates");
+                if(pointsList.size() == 1) {
+                    final List<List<Double>> poly = pointsList.get(0);
+                    for (final List<Double> point : poly) {
+                        coords.add(new Coordinate(point.get(0), point.get(1)));
+                    }
+                    geo = geoFact.createPolygon(coords.toCoordinateArray());
+                } else {
+                    final List<List<Double>> first = pointsList.get(0);
+                    final CoordinateList shellCoords = new CoordinateList();
+                    for (final List<Double> point : pointsList.get(0)) {
+                        shellCoords.add(new Coordinate(point.get(0), point.get(1)));
+                    }
+                    final LinearRing shell = geoFact.createLinearRing(shellCoords.toCoordinateArray());
+
+                    final List<List<List<Double>>> holesPoints = pointsList.subList(1, pointsList.size() - 1);
+                    final LinearRing[] holes = new LinearRing[holesPoints.size()];
+                    for(int ii = 0; ii < holes.length; ii++) {
+                        final List<List<Double>> holePoints = holesPoints.get(ii);
+                        final CoordinateList shells = new CoordinateList();
+                        for (final List<Double> point : pointsList.get(0)) {
+                            shells.add(new Coordinate(point.get(0), point.get(1)));
+                        }
+                        holes[ii] = geoFact.createLinearRing(shells.toCoordinateArray());
+                    }
+                    geo = geoFact.createPolygon(shell,
+                            holes);
+                }
             }
             builder.setGeometry(geo);
         }
@@ -143,5 +168,4 @@ public class EventDocumentConverter implements DocumentConverter<Event>{
         }
         return builder.build();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
index ab44ffe..bc79de4 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
@@ -74,13 +74,13 @@ public class GeoTemporalMongoDBStorageStrategy extends IndexingMongoDBStorageStr
     private final GeoMongoDBStorageStrategy geoStrategy;
 
     public GeoTemporalMongoDBStorageStrategy() {
-        geoStrategy = new GeoMongoDBStorageStrategy(0);
+        geoStrategy = new GeoMongoDBStorageStrategy(0.0);
         temporalStrategy = new TemporalMongoDBStorageStrategy();
     }
 
     @Override
     public void createIndices(final DBCollection coll){
-        coll.createIndex(GEO_KEY);
+        coll.createIndex(new BasicDBObject(GEO_KEY, "2dsphere"));
         coll.createIndex(TIME_KEY);
     }
 
@@ -124,7 +124,11 @@ public class GeoTemporalMongoDBStorageStrategy extends IndexingMongoDBStorageStr
             try {
                 final Statement statement = RyaToRdfConversions.convertStatement(ryaStatement);
                 final Geometry geo = GeoParseUtils.getGeometry(statement);
-                builder.add(GEO_KEY, geoStrategy.getCorrespondingPoints(geo));
+                if (geo.getNumPoints() > 1) {
+                    builder.add(GEO_KEY, geoStrategy.getCorrespondingPoints(geo));
+                } else {
+                    builder.add(GEO_KEY, geoStrategy.getDBPoint(geo));
+                }
             } catch (final ParseException e) {
                 LOG.error("Could not create geometry for statement " + ryaStatement, e);
                 return null;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
index 8b2ebc3..634359f 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
@@ -27,14 +27,17 @@ import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RyaToRdfConversions;
 import org.apache.rya.indexing.accumulo.geo.GeoParseUtils;
 import org.apache.rya.indexing.mongodb.IndexingMongoDBStorageStrategy;
+import org.bson.Document;
 import org.openrdf.model.Statement;
 import org.openrdf.query.MalformedQueryException;
 
 import com.mongodb.BasicDBObject;
+import com.mongodb.BasicDBObjectBuilder;
 import com.mongodb.DBCollection;
 import com.mongodb.DBObject;
 import com.vividsolutions.jts.geom.Coordinate;
 import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.Point;
 import com.vividsolutions.jts.geom.Polygon;
 import com.vividsolutions.jts.io.ParseException;
 import com.vividsolutions.jts.io.WKTReader;
@@ -54,8 +57,12 @@ public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
             public String getKeyword() {
                 return "$geoWithin";
             }
-        },
-        EQUALS {
+        }, EQUALS {
+            @Override
+            public String getKeyword() {
+                return "$near";
+            }
+        }, NEAR {
             @Override
             public String getKeyword() {
                 return "$near";
@@ -69,52 +76,84 @@ public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
         private final GeoQueryType queryType;
         private final Geometry geo;
 
+        private final Double maxDistance;
+        private final Double minDistance;
+
         public GeoQuery(final GeoQueryType queryType, final Geometry geo) {
+            this(queryType, geo, 0, 0);
+        }
+
+        public GeoQuery(final GeoQueryType queryType, final Geometry geo, final double maxDistance,
+                final double minDistance) {
             this.queryType = queryType;
             this.geo = geo;
+            this.maxDistance = maxDistance;
+            this.minDistance = minDistance;
         }
 
         public GeoQueryType getQueryType() {
             return queryType;
         }
+
         public Geometry getGeo() {
             return geo;
         }
+
+        public Double getMaxDistance() {
+            return maxDistance;
+        }
+
+        public Double getMinDistance() {
+            return minDistance;
+        }
     }
 
-    private final double maxDistance;
+    private final Double maxDistance;
 
-    public GeoMongoDBStorageStrategy(final double maxDistance) {
+    public GeoMongoDBStorageStrategy(final Double maxDistance) {
         this.maxDistance = maxDistance;
     }
 
     @Override
     public void createIndices(final DBCollection coll){
-        coll.createIndex("{" + GEO + " : \"2dsphere\"" );
+        coll.createIndex(new BasicDBObject(GEO, "2dsphere"));
     }
 
     public DBObject getQuery(final GeoQuery queryObj) throws MalformedQueryException {
         final Geometry geo = queryObj.getGeo();
         final GeoQueryType queryType = queryObj.getQueryType();
-        if(queryType != GeoQueryType.EQUALS && !(geo instanceof Polygon)) {
+        if (queryType == GeoQueryType.WITHIN && !(geo instanceof Polygon)) {
             //They can also be applied to MultiPolygons, but those are not supported either.
             throw new MalformedQueryException("Mongo Within operations can only be performed on Polygons.");
+        } else if(queryType == GeoQueryType.NEAR && !(geo instanceof Point)) {
+            //They can also be applied to Point, but those are not supported either.
+            throw new MalformedQueryException("Mongo near operations can only be performed on Points.");
         }
 
         BasicDBObject query;
         if (queryType.equals(GeoQueryType.EQUALS)){
-            final List<double[]> points = getCorrespondingPoints(geo);
-            if (points.size() == 1){
+            if(geo.getNumPoints() == 1) {
                 final List circle = new ArrayList();
-                circle.add(points.get(0));
+                circle.add(getPoint(geo));
                 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);
+                query = new BasicDBObject(GEO, getCorrespondingPoints(geo));
             }
+        } else if(queryType.equals(GeoQueryType.NEAR)) {
+            final BasicDBObject geoDoc = new BasicDBObject("$geometry", getDBPoint(geo));
+            if(queryObj.getMaxDistance() != 0) {
+                geoDoc.append("$maxDistance", queryObj.getMaxDistance());
+            }
+
+            if(queryObj.getMinDistance() != 0) {
+                geoDoc.append("$minDistance", queryObj.getMinDistance());
+            }
+            query = new BasicDBObject(GEO, new BasicDBObject(queryType.getKeyword(), geoDoc));
         } else {
-            query = new BasicDBObject(GEO, new BasicDBObject(queryType.getKeyword(), new BasicDBObject("$polygon", getCorrespondingPoints(geo))));
+            final BasicDBObject geoDoc = new BasicDBObject("$geometry", getCorrespondingPoints(geo));
+            query = new BasicDBObject(GEO, new BasicDBObject(queryType.getKeyword(), geoDoc));
         }
 
         return query;
@@ -132,7 +171,11 @@ public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
                 return null;
             }
             final BasicDBObject base = (BasicDBObject) super.serialize(ryaStatement);
-            base.append(GEO, getCorrespondingPoints(geo));
+            if (geo.getNumPoints() > 1) {
+                base.append(GEO, getCorrespondingPoints(geo));
+            } else {
+                base.append(GEO, getDBPoint(geo));
+            }
             return base;
         } catch(final ParseException e) {
             LOG.error("Could not create geometry for statement " + ryaStatement, e);
@@ -140,13 +183,65 @@ public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
         }
     }
 
-    public 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
-            });
+    public Document getCorrespondingPoints(final Geometry geo) {
+        //Polygons must be a 3 dimensional array.
+
+        //polygons must be a closed loop
+        final Document geoDoc = new Document();
+        if (geo instanceof Polygon) {
+            final Polygon poly = (Polygon) geo;
+            final List<List<List<Double>>> DBpoints = new ArrayList<>();
+
+            // outer shell of the polygon
+            final List<List<Double>> ring = new ArrayList<>();
+            for (final Coordinate coord : poly.getExteriorRing().getCoordinates()) {
+                ring.add(getPoint(coord));
+            }
+            DBpoints.add(ring);
+
+            // each hold in the polygon
+            for (int ii = 0; ii < poly.getNumInteriorRing(); ii++) {
+                final List<List<Double>> holeCoords = new ArrayList<>();
+                for (final Coordinate coord : poly.getInteriorRingN(ii).getCoordinates()) {
+                    holeCoords.add(getPoint(coord));
+                }
+                DBpoints.add(holeCoords);
+            }
+            geoDoc.append("coordinates", DBpoints)
+                  .append("type", "Polygon");
+        } else {
+            final List<List<Double>> points = getPoints(geo);
+            geoDoc.append("coordinates", points)
+                  .append("type", "LineString");
+        }
+        return geoDoc;
+    }
+
+    private List<List<Double>> getPoints(final Geometry geo) {
+        final List<List<Double>> points = new ArrayList<>();
+        for (final Coordinate coord : geo.getCoordinates()) {
+            points.add(getPoint(coord));
         }
         return points;
     }
+
+    public Document getDBPoint(final Geometry geo) {
+        return new Document()
+            .append("coordinates", getPoint(geo))
+            .append("type", "Point");
+    }
+
+    private List<Double> getPoint(final Coordinate coord) {
+        final List<Double> point = new ArrayList<>();
+        point.add(coord.x);
+        point.add(coord.y);
+        return point;
+    }
+
+    private List<Double> getPoint(final Geometry geo) {
+        final List<Double> point = new ArrayList<>();
+        point.add(geo.getCoordinate().x);
+        point.add(geo.getCoordinate().y);
+        return point;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java
index 45da13a..2abee76 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java
@@ -20,12 +20,14 @@ package org.apache.rya.indexing.mongodb.geo;
 
 import static org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.EQUALS;
 import static org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.INTERSECTS;
+import static org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.NEAR;
 import static org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQueryType.WITHIN;
 
 import org.apache.log4j.Logger;
 import org.apache.rya.indexing.GeoIndexer;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.geo.GeoTupleSet.GeoSearchFunctionFactory.NearQuery;
 import org.apache.rya.indexing.mongodb.AbstractMongoIndexer;
 import org.apache.rya.indexing.mongodb.geo.GeoMongoDBStorageStrategy.GeoQuery;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
@@ -111,6 +113,27 @@ public class MongoGeoIndexer extends AbstractMongoIndexer<GeoMongoDBStorageStrat
     }
 
     @Override
+    public CloseableIteration<Statement, QueryEvaluationException> queryNear(final NearQuery query, final StatementConstraints constraints) {
+        double maxDistance = 0;
+        double minDistance = 0;
+        if (query.getMaxDistance().isPresent()) {
+            maxDistance = query.getMaxDistance().get();
+        }
+
+        if (query.getMinDistance().isPresent()) {
+            minDistance = query.getMinDistance().get();
+        }
+
+        try {
+            final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(NEAR, query.getGeometry(), maxDistance, minDistance));
+            return withConstraints(constraints, queryObj);
+        } catch (final MalformedQueryException e) {
+            logger.error(e.getMessage(), e);
+            return null;
+        }
+    }
+
+    @Override
     public CloseableIteration<Statement, QueryEvaluationException> queryContains(
             final Geometry query, final StatementConstraints constraints) {
         throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
deleted file mode 100644
index aa2a129..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
+++ /dev/null
@@ -1,360 +0,0 @@
-package org.apache.rya.indexing.mongodb.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 org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.indexing.GeoIndexer;
-import org.apache.rya.indexing.IndexingExpr;
-import org.apache.rya.indexing.IteratorFactory;
-import org.apache.rya.indexing.SearchFunction;
-import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.indexing.accumulo.geo.GeoTupleSet;
-import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
-
-public class MongoGeoTupleSet extends ExternalTupleSet {
-
-    private Configuration conf;
-    private GeoIndexer geoIndexer;
-    private IndexingExpr filterInfo;
-   
-
-    public MongoGeoTupleSet(IndexingExpr filterInfo, GeoIndexer geoIndexer) {
-        this.filterInfo = filterInfo;
-        this.geoIndexer = geoIndexer;
-        this.conf = geoIndexer.getConf();
-    }
-
-    @Override
-    public Set<String> getBindingNames() {
-        return filterInfo.getBindingNames();
-    }
-
-    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(this.getBindingNames()).replaceAll("\\s+", " ");
-    }
-    
-    
-    
-    @Override
-    public boolean equals(Object other) {
-        if (other == this) {
-            return true;
-        }
-        if (!(other instanceof MongoGeoTupleSet)) {
-            return false;
-        }
-        MongoGeoTupleSet arg = (MongoGeoTupleSet) other;
-        return this.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(BindingSet bindings)
-            throws QueryEvaluationException {
-        
-      
-        URI funcURI = filterInfo.getFunction();
-        SearchFunction searchFunction = (new MongoGeoSearchFunctionFactory(conf)).getSearchFunction(funcURI);
-        if(filterInfo.getArguments().length > 1) {
-            throw new IllegalArgumentException("Index functions do not support more than two arguments.");
-        }
-        
-        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 MongoGeoSearchFunctionFactory {
-        
-        Configuration conf;
-        
-        private final Map<URI, SearchFunction> SEARCH_FUNCTION_MAP = Maps.newHashMap();
-
-        public MongoGeoSearchFunctionFactory(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 (QueryEvaluationException e) {
-                e.printStackTrace();
-            }
-
-            return geoFunc;
-        }
-
-        private SearchFunction getSearchFunctionInternal(final URI searchFunction) throws QueryEvaluationException {
-            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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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(String queryText,
-                    StatementConstraints contraints) throws QueryEvaluationException {
-                try {
-                    WKTReader reader = new WKTReader();
-                    Geometry geometry = reader.read(queryText);
-                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
-                            geometry, contraints);
-                    return statements;
-                } catch (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/5e26bbe2/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
index edce1ec..4a31599 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
@@ -26,11 +26,9 @@ import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.IndexingExpr;
 import org.apache.rya.indexing.IndexingFunctionRegistry;
 import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
-import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer;
-import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.GeoPolicy;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.TemporalPolicy;
-import org.apache.rya.indexing.geotemporal.mongo.GeoTemporalMongoDBStorageStrategy;
+import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -99,12 +97,15 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
         final DBObject actual = adapter.getFilterQuery(geoFilters, temporalFilters);
         final String expectedString =
             "{ "
-              + "\"location\" : {"
-                + "\"$geoWithin\" : {"
-                  + "\"$polygon\" : [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]"
+            + "\"location\" : { "
+              + "\"$geoWithin\" : { "
+                + "\"$geometry\" : { "
+                  + "\"coordinates\" : [ [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]] , "
+                  + "\"type\" : \"Polygon\""
                 + "}"
               + "}"
-            + "}";
+            + "}"
+          + "}";
         final DBObject expected = (DBObject) JSON.parse(expectedString);
         assertEqualMongo(expected, actual);
     }
@@ -136,18 +137,24 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
               }
               final List<IndexingExpr> temporalFilters = new ArrayList<>();
               final DBObject actual = adapter.getFilterQuery(geoFilters, temporalFilters);
+
               final String expectedString =
                   "{ "
-                    + "\"$and\" : [{"
-                      + "\"location\" : [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]"
-                      + "}, {"
-                      + "\"location\" : {"
-                        + "\"$geoIntersects\" : {"
-                          + "\"$polygon\" : [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]"
-                        + "}"
+                  + "\"$and\" : [ { "
+                    + "\"location\" : {"
+                      + " \"coordinates\" : [ [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]] ,"
+                      + " \"type\" : \"Polygon\""
+                    + "}"
+                  + "} , { "
+                  + "\"location\" : { "
+                    + "\"$geoIntersects\" : {"
+                      + " \"$geometry\" : {"
+                        + " \"coordinates\" : [ [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]] ,"
+                        + " \"type\" : \"Polygon\""
                       + "}"
-                    + "}]"
-                  + "}";
+                    + "}"
+                  + "}"
+                + "}]}";
               final DBObject expected = (DBObject) JSON.parse(expectedString);
               assertEqualMongo(expected, actual);
     }
@@ -257,14 +264,18 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
               final DBObject actual = adapter.getFilterQuery(geoFilters, temporalFilters);
               final String expectedString =
               "{ "
-              + "\"$and\" : [{"
-                + "\"location\" : {"
-                  + "\"$geoWithin\" : {"
-                    + "\"$polygon\" : [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]"
-                  + "},"
-                + "}}, {"
-                + "\"instant\" : {"
-                  + "\"$gt\" : {"
+              + "\"$and\" : [ { "
+                + "\"location\" : { "
+                  + "\"$geoWithin\" : { "
+                    + "\"$geometry\" : { "
+                      + "\"coordinates\" : [ [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]] , "
+                      + "\"type\" : \"Polygon\""
+                    + "}"
+                  + "}"
+                + "}"
+              + "} , { "
+                + "\"instant\" : { "
+                  + "\"$gt\" : { "
                     + "\"$date\" : \"2015-12-30T12:00:00.000Z\""
                   + "}"
                 + "}"
@@ -306,32 +317,37 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
               }
               final DBObject actual = adapter.getFilterQuery(geoFilters, temporalFilters);
               final String expectedString =
-              "{ "
-              + "\"$and\" : [{"
-                + "\"$and\" : [{"
-                  + "\"location\" : [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]"
-                  + "}, {"
-                  + "\"location\" : {"
-                    + "\"$geoWithin\" : {"
-                      + "\"$polygon\" : [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]"
-                    + "}"
-                  + "}"
-                + "}]"
-              + "},{"
-                + "\"$and\" : [{"
-                  + "\"instant\" : {"
-                    + "\"$lt\" : {"
-                      + "\"$date\" : \"1970-01-01T00:00:00.000Z\""
-                    + "},"
-                    + "}"
-                  + "}, {"
-                    + "\"instant\" : {"
-                      + "\"$date\" : \"1970-01-01T00:00:01.000Z\""
-                    + "}"
-                  + "}]"
-                + "}"
-              + "]"
-            + "}";
+                  "{ "
+                  + "\"$and\" : [ { "
+                    + "\"$and\" : [ { "
+                      + "\"location\" : { "
+                        + "\"coordinates\" : [ [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]] , "
+                        + "\"type\" : \"Polygon\""
+                      + "}"
+                    + "} , { "
+                      + "\"location\" : { "
+                        + "\"$geoWithin\" : { "
+                          + "\"$geometry\" : { "
+                            + "\"coordinates\" : [ [ [ -3.0 , -2.0] , [ -3.0 , 2.0] , [ 1.0 , 2.0] , [ 1.0 , -2.0] , [ -3.0 , -2.0]]] , "
+                            + "\"type\" : \"Polygon\""
+                         + "}"
+                       + "}"
+                     + "}"
+                   + "}]"
+                 + "} , { "
+                   + "\"$and\" : [ { "
+                     + "\"instant\" : { "
+                       + "\"$lt\" : { "
+                         + "\"$date\" : \"1970-01-01T00:00:00.000Z\""
+                       + "}"
+                     + "}"
+                   + "} , { "
+                     + "\"instant\" : { "
+                       + "\"$date\" : \"1970-01-01T00:00:01.000Z\""
+                     + "}"
+                   + "}]"
+                 + "}]"
+               + "}";
               final DBObject expected = (DBObject) JSON.parse(expectedString);
               assertEqualMongo(expected, actual);
     }
@@ -367,24 +383,26 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
         }
         final DBObject actual = adapter.getFilterQuery(geoFilters, temporalFilters);
         final String expectedString =
-              "{ "
-              + "\"$and\" : [{"
-                  + "\"location\" : [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]"
-                + "},{"
-                + "\"$and\" : [{"
-                  + "\"instant\" : {"
-                    + "\"$gt\" : {"
-                      + "\"$date\" : \"1970-01-01T00:00:01.000Z\""
-                    + "},"
-                    + "}"
-                  + "}, {"
-                    + "\"instant\" : {"
-                      + "\"$date\" : \"1970-01-01T00:00:00.000Z\""
-                    + "}"
-                  + "}]"
+            "{ "
+            + "\"$and\" : [ { "
+              + "\"location\" : { "
+                + "\"coordinates\" : [ [ [ -4.0 , -3.0] , [ -4.0 , 3.0] , [ 2.0 , 3.0] , [ 2.0 , -3.0] , [ -4.0 , -3.0]]] , "
+                + "\"type\" : \"Polygon\""
+              + "}"
+            + "} , { "
+              + "\"$and\" : [ { "
+                + "\"instant\" : { "
+                  + "\"$gt\" : { "
+                    + "\"$date\" : \"1970-01-01T00:00:01.000Z\""
+                  + "}"
                 + "}"
-              + "]"
-            + "}";
+              + "} , { "
+                + "\"instant\" : { "
+                  + "\"$date\" : \"1970-01-01T00:00:00.000Z\""
+                + "}"
+              + "}]"
+            + "}]"
+          + "}";
         final DBObject expected = (DBObject) JSON.parse(expectedString);
         assertEqualMongo(expected, actual);
     }
@@ -402,10 +420,13 @@ public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
         Statement statement = new ContextStatementImpl(subject, predicate, object, context);
         DBObject actual = adapter.serialize(RdfToRyaConversions.convertStatement(statement));
         String expectedString =
-                "{"
-                  +"_id : -852305321, "
-                  +"location : [ [ -77.03524 , 38.889468]]"
-              + "}";
+            "{ "
+            + "\"_id\" : -852305321 , "
+            + "\"location\" : { "
+              + "\"coordinates\" : [ -77.03524 , 38.889468] , "
+              + "\"type\" : \"Point\""
+            + "}"
+          + "}";
         DBObject expected = (DBObject) JSON.parse(expectedString);
         assertEqualMongo(expected, actual);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e26bbe2/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIT.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIT.java
deleted file mode 100644
index 218b454..0000000
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIT.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.indexing.mongo;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.resolver.RdfToRyaConversions;
-import org.apache.rya.api.resolver.RyaToRdfConversions;
-import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.indexing.GeoRyaSailFactory;
-import org.apache.rya.indexing.OptionalConfigUtils;
-import org.apache.rya.indexing.TemporalInstant;
-import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.junit.Before;
-import org.junit.Test;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.QueryLanguage;
-import org.openrdf.query.Update;
-import org.openrdf.repository.sail.SailRepository;
-import org.openrdf.repository.sail.SailRepositoryConnection;
-import org.openrdf.sail.Sail;
-
-import com.mongodb.MongoClient;
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.GeometryFactory;
-
-public class MongoGeoIT {
-    private MongoClient client;
-    private Sail sail;
-    private SailRepositoryConnection conn;
-
-    @Before
-    public void before() throws Exception {
-        final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
-            .setMongoCollectionPrefix("rya_")
-            .setMongoDBName("indexerTests")
-            .setUseMongoFreetextIndex(true)
-            .setUseMongoTemporalIndex(true)
-            .setMongoFreeTextPredicates(RDFS.LABEL.stringValue())
-            .setMongoTemporalPredicates("Property:atTime")
-            .build();
-
-        client = MockMongoFactory.newFactory().newMongoClient();
-        indxrConf.setBoolean(OptionalConfigUtils.USE_GEO, true);
-        indxrConf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
-        indxrConf.setBoolean(ConfigUtils.USE_MONGO, true);
-        indxrConf.setMongoClient(client);
-
-        sail = GeoRyaSailFactory.getInstance(indxrConf);
-        conn = new SailRepository(sail).getConnection();
-        conn.begin();
-    }
-
-    @Test
-    public void deleteTest() throws Exception {
-        populateRya();
-
-        assertEquals(8, client.getDatabase("indexerTests").getCollection("rya__triples").count());
-        assertEquals(4, client.getDatabase("indexerTests").getCollection("rya_rya_geo").count());
-        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
-        assertEquals(2, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
-
-        //free text -- remove one from many
-        String delete = "DELETE DATA \n" //
-           + "{\n"
-           + "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
-           + "}";
-        Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-        update.execute();
-
-        // temporal -- remove one from one
-        delete = "DELETE DATA \n" //
-           + "{\n"
-           + "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
-           + "}";
-
-        update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-        update.execute();
-
-        assertEquals(0, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
-        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
-        assertEquals(6, client.getDatabase("indexerTests").getCollection("rya__triples").count());
-    }
-
-    private void populateRya() throws Exception {
-        final ValueFactory VF = new ValueFactoryImpl();
-        // geo 2x2 points
-        final GeometryFactory GF = new GeometryFactory();
-        for (int x = 0; x <= 1; x++) {
-            for (int y = 0; y <= 1; y++) {
-                final Geometry geo = GF.createPoint(new Coordinate(x + .5, y + .5));
-                final RyaStatement stmnt = statement(geo);
-                final Statement statement = RyaToRdfConversions.convertStatement(stmnt);
-                conn.add(statement);
-            }
-        }
-
-        // freetext
-        final URI person = VF.createURI("http://example.org/ontology/Person");
-        String uuid;
-
-        uuid = "urn:people";
-        conn.add(VF.createURI(uuid), RDF.TYPE, person);
-        conn.add(VF.createURI(uuid), RDFS.LABEL, VF.createLiteral("Alice Palace Hose", VF.createURI("http://www.w3.org/2001/XMLSchema#string")));
-
-        uuid = "urn:people";
-        conn.add(VF.createURI(uuid), RDF.TYPE, person);
-        conn.add(VF.createURI(uuid), RDFS.LABEL, VF.createLiteral("Bob Snob Hose", "en"));
-
-        // temporal
-        final TemporalInstant instant = new TemporalInstantRfc3339(1, 2, 3, 4, 5, 6);
-        final URI time = VF.createURI("Property:atTime");
-        conn.add(VF.createURI("foo:time"), VF.createURI("Property:atTime"), VF.createLiteral(instant.toString()));
-    }
-
-    private static RyaStatement statement(final Geometry geo) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("urn:geo");
-        final URI predicate = GeoConstants.GEO_AS_WKT;
-        final Value object = vf.createLiteral(geo.toString(), GeoConstants.XMLSCHEMA_OGC_WKT);
-        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
-    }
-
-}


[3/3] incubator-rya git commit: RYA-323 Enabling Multi Var Geo Functions. Closes #183

Posted by mi...@apache.org.
RYA-323 Enabling Multi Var Geo Functions. Closes #183


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

Branch: refs/heads/master
Commit: 94911ed894e48c52ea5f69af6f1ac41c752c7a3e
Parents: 5e26bbe
Author: Aaron Mihalik <mi...@alum.mit.edu>
Authored: Tue Jul 25 13:43:03 2017 -0400
Committer: Aaron Mihalik <aa...@gmail.com>
Committed: Thu Aug 3 00:28:34 2017 -0400

----------------------------------------------------------------------
 .../org/apache/rya/indexing/IndexingExpr.java   |   6 +-
 .../rya/indexing/IndexingFunctionRegistry.java  |   2 +
 .../apache/rya/indexing/TemporalTupleSet.java   |   3 +-
 .../accumulo/freetext/FreeTextTupleSet.java     |   3 +-
 .../GeoEnabledFilterFunctionOptimizer.java      |   4 +-
 .../indexing/accumulo/geo/GeoParseUtils.java    |   6 +-
 .../rya/indexing/accumulo/geo/GeoTupleSet.java  |  21 +-
 .../GeoTemporalMongoDBStorageStrategy.java      |   5 +-
 .../indexing/mongodb/geo/MongoGeoTupleSet.java  | 361 +++++++++++++++++++
 .../evaluation/QueryJoinOptimizer.java          |  21 +-
 10 files changed, 413 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingExpr.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingExpr.java b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingExpr.java
index 37e98af..f919b18 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingExpr.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingExpr.java
@@ -32,10 +32,10 @@ import com.google.common.collect.Sets;
 public class IndexingExpr {
 
     private final URI function;
-    private final Value[] arguments;
+    private final Object[] arguments;
     private final StatementPattern spConstraint;
 
-    public IndexingExpr(URI function, StatementPattern spConstraint, Value... arguments) {
+    public IndexingExpr(URI function, StatementPattern spConstraint, Object... arguments) {
         this.function = function;
         this.arguments = arguments;
         this.spConstraint = spConstraint;
@@ -45,7 +45,7 @@ public class IndexingExpr {
         return function;
     }
 
-    public Value[] getArguments() {
+    public Object[] getArguments() {
         return arguments;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
index b61af4b..37f7116 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/IndexingFunctionRegistry.java
@@ -115,6 +115,8 @@ public class IndexingFunctionRegistry {
                 return (Var) arg1;
             else if (isUnboundVariable(arg2) && isConstant(arg1))
                 return (Var) arg2;
+            else 
+                return (Var) arg1;
         }
         return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/indexing/src/main/java/org/apache/rya/indexing/TemporalTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/TemporalTupleSet.java b/extras/indexing/src/main/java/org/apache/rya/indexing/TemporalTupleSet.java
index 7cb4e6c..3f20191 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/TemporalTupleSet.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/TemporalTupleSet.java
@@ -8,6 +8,7 @@ import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.joda.time.DateTime;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
+import org.openrdf.model.Value;
 import org.openrdf.model.impl.URIImpl;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.QueryEvaluationException;
@@ -117,7 +118,7 @@ public class TemporalTupleSet extends ExternalTupleSet {
             throw new IllegalArgumentException("Index functions do not support more than two arguments.");
         }
 
-        final String queryText = filterInfo.getArguments()[0].stringValue();
+        final String queryText = ((Value) filterInfo.getArguments()[0]).stringValue();
         return IteratorFactory.getIterator(filterInfo.getSpConstraint(), bindings, queryText, searchFunction);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/freetext/FreeTextTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/freetext/FreeTextTupleSet.java b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/freetext/FreeTextTupleSet.java
index 6b01811..383b969 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/freetext/FreeTextTupleSet.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/accumulo/freetext/FreeTextTupleSet.java
@@ -35,6 +35,7 @@ import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.apache.hadoop.conf.Configuration;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
+import org.openrdf.model.Value;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.QueryEvaluationException;
 import org.openrdf.query.algebra.QueryModelVisitor;
@@ -152,7 +153,7 @@ public class FreeTextTupleSet extends ExternalTupleSet {
             throw new IllegalArgumentException("Index functions do not support more than two arguments.");
         }
 
-        String queryText = filterInfo.getArguments()[0].stringValue();
+        String queryText = ((Value) filterInfo.getArguments()[0]).stringValue();
 
         return IteratorFactory.getIterator(filterInfo.getSpConstraint(), bindings, queryText, searchFunction);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
index bf6b632..b7c49d8 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
@@ -207,13 +207,13 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
     private abstract class AbstractEnhanceVisitor extends QueryModelVisitorBase<RuntimeException> {
         final String matchVar;
         List<URI> func = Lists.newArrayList();
-        List<Value[]> args = Lists.newArrayList();
+        List<Object[]> args = Lists.newArrayList();
 
         public AbstractEnhanceVisitor(final String matchVar) {
             this.matchVar = matchVar;
         }
 
-        protected void addFilter(final URI uri, final Value[] values) {
+        protected void addFilter(final URI uri, final Object[] values) {
             func.add(uri);
             args.add(values);
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoParseUtils.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoParseUtils.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoParseUtils.java
index e8fbc3d..103b241 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoParseUtils.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoParseUtils.java
@@ -126,8 +126,8 @@ public class GeoParseUtils {
      * @param call - The {@link FunctionCall} to match against.
      * @return - The {@link Value}s matched.
      */
-    public static Value[] extractArguments(final String matchName, final FunctionCall call) {
-        final Value args[] = new Value[call.getArgs().size() - 1];
+    public static Object[] extractArguments(final String matchName, final FunctionCall call) {
+        final Object[] args = new Object[call.getArgs().size() - 1];
         int argI = 0;
         for (int i = 0; i != call.getArgs().size(); ++i) {
             final ValueExpr arg = call.getArgs().get(i);
@@ -139,7 +139,7 @@ public class GeoParseUtils {
             } else if (arg instanceof Var && ((Var)arg).hasValue()) {
                 args[argI] = ((Var)arg).getValue();
             } else {
-                throw new IllegalArgumentException("Query error: Found " + arg + ", expected a Literal, BNode or URI");
+                args[argI] = arg;
             }
             ++argI;
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
index 3f4be9c..8cdeb5c 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/accumulo/geo/GeoTupleSet.java
@@ -20,6 +20,7 @@ import org.openrdf.model.URI;
 import org.openrdf.model.Value;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.Var;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Maps;
@@ -123,7 +124,15 @@ public class GeoTupleSet extends ExternalTupleSet {
         final URI funcURI = filterInfo.getFunction();
         final SearchFunction searchFunction = new GeoSearchFunctionFactory(conf, geoIndexer).getSearchFunction(funcURI);
 
-        String queryText = filterInfo.getArguments()[0].stringValue();
+        String queryText;
+        Object arg = filterInfo.getArguments()[0];
+        if (arg instanceof Value) {
+            queryText = ((Value) arg).stringValue();
+        } else if (arg instanceof Var) {
+            queryText = bindings.getBinding(((Var) arg).getName()).getValue().stringValue();
+        } else {
+            throw new IllegalArgumentException("Query text was not resolved");
+        }
 
         if(funcURI.equals(GeoConstants.GEO_SF_NEAR)) {
             if (filterInfo.getArguments().length > 3) {
@@ -131,8 +140,14 @@ public class GeoTupleSet extends ExternalTupleSet {
             }
 
             final List<String> valueList = new ArrayList<>();
-            for (final Value val : filterInfo.getArguments()) {
-                valueList.add(val.stringValue());
+            for (final Object val : filterInfo.getArguments()) {
+                if (val instanceof Value) {
+                    valueList.add(((Value)val).stringValue());
+                } else if (val instanceof Var) {
+                    valueList.add(bindings.getBinding(((Var) val).getName()).getValue().stringValue());
+                } else {
+                    throw new IllegalArgumentException("Query text was not resolved");
+                }
             }
             queryText = Strings.join(valueList, NEAR_DELIM);
         } else if (filterInfo.getArguments().length > 1) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
index bc79de4..6e8ed99 100644
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategy.java
@@ -49,6 +49,7 @@ import org.apache.rya.indexing.mongodb.temporal.TemporalMongoDBStorageStrategy;
 import org.joda.time.DateTime;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
+import org.openrdf.model.Value;
 import org.openrdf.query.MalformedQueryException;
 
 import com.mongodb.BasicDBObject;
@@ -144,7 +145,7 @@ public class GeoTemporalMongoDBStorageStrategy extends IndexingMongoDBStorageStr
         geoFilters.forEach(filter -> {
             final GeoPolicy policy = GeoPolicy.fromURI(filter.getFunction());
             final WKTReader reader = new WKTReader();
-            final String geoStr = filter.getArguments()[0].stringValue();
+            final String geoStr = ((Value) filter.getArguments()[0]).stringValue();
             try {
                 //This method is what is used in the GeoIndexer.
                 final Geometry geo = reader.read(geoStr);
@@ -160,7 +161,7 @@ public class GeoTemporalMongoDBStorageStrategy extends IndexingMongoDBStorageStr
         final List<DBObject> objs = new ArrayList<>();
         temporalFilters.forEach(filter -> {
             final TemporalPolicy policy = TemporalPolicy.fromURI(filter.getFunction());
-            final String timeStr = filter.getArguments()[0].stringValue();
+            final String timeStr = ((Value) filter.getArguments()[0]).stringValue();
             final Matcher matcher = TemporalInstantRfc3339.PATTERN.matcher(timeStr);
             if(matcher.find()) {
                 final TemporalInterval interval = TemporalInstantRfc3339.parseInterval(timeStr);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94911ed8/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
new file mode 100644
index 0000000..c564d02
--- /dev/null
+++ b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
@@ -0,0 +1,361 @@
+package org.apache.rya.indexing.mongodb.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.model.Value;
+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 org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.indexing.GeoIndexer;
+import org.apache.rya.indexing.IndexingExpr;
+import org.apache.rya.indexing.IteratorFactory;
+import org.apache.rya.indexing.SearchFunction;
+import org.apache.rya.indexing.StatementConstraints;
+import org.apache.rya.indexing.accumulo.geo.GeoTupleSet;
+import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
+
+public class MongoGeoTupleSet extends ExternalTupleSet {
+
+    private Configuration conf;
+    private GeoIndexer geoIndexer;
+    private IndexingExpr filterInfo;
+   
+
+    public MongoGeoTupleSet(IndexingExpr filterInfo, GeoIndexer geoIndexer) {
+        this.filterInfo = filterInfo;
+        this.geoIndexer = geoIndexer;
+        this.conf = geoIndexer.getConf();
+    }
+
+    @Override
+    public Set<String> getBindingNames() {
+        return filterInfo.getBindingNames();
+    }
+
+    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(this.getBindingNames()).replaceAll("\\s+", " ");
+    }
+    
+    
+    
+    @Override
+    public boolean equals(Object other) {
+        if (other == this) {
+            return true;
+        }
+        if (!(other instanceof MongoGeoTupleSet)) {
+            return false;
+        }
+        MongoGeoTupleSet arg = (MongoGeoTupleSet) other;
+        return this.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(BindingSet bindings)
+            throws QueryEvaluationException {
+        
+      
+        URI funcURI = filterInfo.getFunction();
+        SearchFunction searchFunction = (new MongoGeoSearchFunctionFactory(conf)).getSearchFunction(funcURI);
+        if(filterInfo.getArguments().length > 1) {
+            throw new IllegalArgumentException("Index functions do not support more than two arguments.");
+        }
+
+        String queryText = ((Value) 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 MongoGeoSearchFunctionFactory {
+        
+        Configuration conf;
+        
+        private final Map<URI, SearchFunction> SEARCH_FUNCTION_MAP = Maps.newHashMap();
+
+        public MongoGeoSearchFunctionFactory(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 (QueryEvaluationException e) {
+                e.printStackTrace();
+            }
+
+            return geoFunc;
+        }
+
+        private SearchFunction getSearchFunctionInternal(final URI searchFunction) throws QueryEvaluationException {
+            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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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(String queryText,
+                    StatementConstraints contraints) throws QueryEvaluationException {
+                try {
+                    WKTReader reader = new WKTReader();
+                    Geometry geometry = reader.read(queryText);
+                    CloseableIteration<Statement, QueryEvaluationException> statements = geoIndexer.queryWithin(
+                            geometry, contraints);
+                    return statements;
+                } catch (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/94911ed8/sail/src/main/java/org/apache/rya/rdftriplestore/evaluation/QueryJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/evaluation/QueryJoinOptimizer.java b/sail/src/main/java/org/apache/rya/rdftriplestore/evaluation/QueryJoinOptimizer.java
index 2cb02ba..b5d5c43 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/evaluation/QueryJoinOptimizer.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/evaluation/QueryJoinOptimizer.java
@@ -1,5 +1,14 @@
 package org.apache.rya.rdftriplestore.evaluation;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.rya.rdftriplestore.utils.DefaultStatistics;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -23,14 +32,16 @@ package org.apache.rya.rdftriplestore.evaluation;
 
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.Dataset;
-import org.openrdf.query.algebra.*;
+import org.openrdf.query.algebra.Join;
+import org.openrdf.query.algebra.LeftJoin;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.Var;
 import org.openrdf.query.algebra.evaluation.QueryOptimizer;
 import org.openrdf.query.algebra.evaluation.impl.EvaluationStatistics;
 import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
 import org.openrdf.query.algebra.helpers.StatementPatternCollector;
 
-import java.util.*;
-
 /**
  * A query optimizer that re-orders nested Joins.
  *
@@ -56,7 +67,9 @@ public class QueryJoinOptimizer implements QueryOptimizer {
      * @param tupleExpr
      */
     public void optimize(TupleExpr tupleExpr, Dataset dataset, BindingSet bindings) {
-        tupleExpr.visit(new JoinVisitor());
+        if (!(statistics instanceof DefaultStatistics)) {
+            tupleExpr.visit(new JoinVisitor());
+        }
     }
 
     protected class JoinVisitor extends QueryModelVisitorBase<RuntimeException> {