You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@rya.apache.org by mi...@apache.org on 2015/12/22 17:50:20 UTC

[56/56] [abbrv] incubator-rya git commit: RYA-10 reusing existing collection for geo queries

RYA-10 reusing existing collection for geo queries


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

Branch: refs/heads/master
Commit: 7743a42a5febe440107e037aadc3753643708e83
Parents: 1007611
Author: pujav65 <pu...@gmail.com>
Authored: Mon Dec 21 22:23:34 2015 -0500
Committer: Aaron Mihalik <mi...@alum.mit.edu>
Committed: Tue Dec 22 11:48:32 2015 -0500

----------------------------------------------------------------------
 .../dao/SimpleMongoDBStorageStrategy.java       |  18 +-
 .../java/mvm/rya/mongodb/MongoDBRyaDAOTest.java |  18 +
 .../indexing/mongodb/AbstractMongoIndexer.java  |   5 -
 .../mongodb/GeoMongoDBStorageStrategy.java      |  51 +--
 .../rya/indexing/mongodb/MongoGeoIndexer.java   | 423 ++++++++++---------
 .../indexing/mongo/MongoGeoIndexerSfTest.java   | 306 ++++++++++++++
 .../rya/indexing/mongo/MongoGeoIndexerTest.java | 390 +++++++++++++++++
 7 files changed, 958 insertions(+), 253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/dao/mongodb.rya/src/main/java/mvm/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/mvm/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java b/dao/mongodb.rya/src/main/java/mvm/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
index 3ecc0dc..74c8366 100644
--- a/dao/mongodb.rya/src/main/java/mvm/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
+++ b/dao/mongodb.rya/src/main/java/mvm/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
@@ -38,13 +38,13 @@ import com.mongodb.DBObject;
 
 public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy {
 
-	private static final String ID = "_id";
-	private static final String OBJECT_TYPE = "objectType";
-	private static final String CONTEXT = "context";
-	private static final String PREDICATE = "predicate";
-	private static final String OBJECT = "object";
-	private static final String SUBJECT = "subject";
-	private ValueFactoryImpl factory = new ValueFactoryImpl();
+	protected static final String ID = "_id";
+	protected static final String OBJECT_TYPE = "objectType";
+	protected static final String CONTEXT = "context";
+	protected static final String PREDICATE = "predicate";
+	protected static final String OBJECT = "object";
+	protected static final String SUBJECT = "subject";
+	protected ValueFactoryImpl factory = new ValueFactoryImpl();
 
 
 	public SimpleMongoDBStorageStrategy() {
@@ -115,6 +115,10 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy {
 
 	@Override
 	public DBObject serialize(RyaStatement statement){
+		return serializeInternal(statement);		
+	}
+
+	public BasicDBObject serializeInternal(RyaStatement statement){
 		String context = "";
 		if (statement.getContext() != null){
 			context = statement.getContext().getData();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/dao/mongodb.rya/src/test/java/mvm/rya/mongodb/MongoDBRyaDAOTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/mvm/rya/mongodb/MongoDBRyaDAOTest.java b/dao/mongodb.rya/src/test/java/mvm/rya/mongodb/MongoDBRyaDAOTest.java
index 3d900b0..86c01e2 100644
--- a/dao/mongodb.rya/src/test/java/mvm/rya/mongodb/MongoDBRyaDAOTest.java
+++ b/dao/mongodb.rya/src/test/java/mvm/rya/mongodb/MongoDBRyaDAOTest.java
@@ -1,4 +1,22 @@
 package mvm.rya.mongodb;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/AbstractMongoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/AbstractMongoIndexer.java b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/AbstractMongoIndexer.java
index 4a708ab..0f8202c 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/AbstractMongoIndexer.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/AbstractMongoIndexer.java
@@ -61,11 +61,6 @@ public abstract class AbstractMongoIndexer implements RyaSecondaryIndexer {
     }
 
     @Override
-    public void deleteStatement(RyaStatement stmt) throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
     public void dropGraph(RyaURI... graphs) {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/GeoMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/GeoMongoDBStorageStrategy.java b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/GeoMongoDBStorageStrategy.java
index 0355225..c21f574 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/GeoMongoDBStorageStrategy.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/GeoMongoDBStorageStrategy.java
@@ -27,9 +27,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.resolver.RdfToRyaConversions;
 import mvm.rya.indexing.StatementContraints;
 import mvm.rya.indexing.accumulo.StatementSerializer;
 import mvm.rya.indexing.accumulo.geo.GeoParseUtils;
+import mvm.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
 
 import org.apache.commons.codec.binary.Hex;
 import org.openrdf.model.Statement;
@@ -44,14 +47,9 @@ import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.io.ParseException;
 import com.vividsolutions.jts.io.WKTReader;
 
-public class GeoMongoDBStorageStrategy {
+public class GeoMongoDBStorageStrategy extends SimpleMongoDBStorageStrategy{
 
-	private static final String ID = "_id";
 	private static final String GEO = "location";
-	private static final String CONTEXT = "context";
-	private static final String PREDICATE = "predicate";
-	private static final String OBJECT = "object";
-	private static final String SUBJECT = "subject";
 	public enum GeoQueryType {
 		INTERSECTS {
 			public String getKeyword() {
@@ -124,21 +122,6 @@ public class GeoMongoDBStorageStrategy {
 		return query;
 	}
 
-
-	public Statement deserializeDBObject(DBObject queryResult) {
-		Map result = queryResult.toMap();
-		String subject = (String) result.get(SUBJECT);
-		String object = (String) result.get(OBJECT);
-		String predicate = (String) result.get(PREDICATE);
-		String context = (String) result.get(CONTEXT);
-		if (!context.isEmpty()){
-			return StatementSerializer.readStatement(subject, predicate, object, context);			
-		}
-		return StatementSerializer.readStatement(subject, predicate, object);
-	}
-	
-	
-
 	public DBObject serialize(Statement statement) throws ParseException{
 		// if the object is wkt, then try to index it
         // write the statement data to the fields
@@ -146,28 +129,10 @@ public class GeoMongoDBStorageStrategy {
         if(geo == null || geo.isEmpty() || !geo.isValid()) {
             throw new ParseException("Could not create geometry for statement " + statement);
         }
- 		
-		String context = "";
-		if (statement.getContext() != null){
-			context = StatementSerializer.writeContext(statement);
-		}
-		String id = StatementSerializer.writeSubject(statement) + " " + 
-				StatementSerializer.writePredicate(statement) + " " +  StatementSerializer.writeObject(statement) + " " + context;
-		byte[] bytes = id.getBytes();
-		try {
-			MessageDigest digest = MessageDigest.getInstance("SHA-1");
-			bytes = digest.digest(bytes);
-		} catch (NoSuchAlgorithmException e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		}
-		BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))
-		.append(GEO, getCorrespondingPoints(geo))
-		.append(SUBJECT, StatementSerializer.writeSubject(statement))
-	    .append(PREDICATE, StatementSerializer.writePredicate(statement))
-	    .append(OBJECT,  StatementSerializer.writeObject(statement))
-	    .append(CONTEXT, context);
-		return doc;
+ 		RyaStatement ryaStatement = RdfToRyaConversions.convertStatement(statement);
+ 		BasicDBObject base = (BasicDBObject) super.serialize(ryaStatement);
+ 		base.append(GEO, getCorrespondingPoints(geo));	
+		return base;
 		
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/MongoGeoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/MongoGeoIndexer.java b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/MongoGeoIndexer.java
index c36b125..199883f 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/MongoGeoIndexer.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/mongodb/MongoGeoIndexer.java
@@ -19,7 +19,6 @@ package mvm.rya.indexing.mongodb;
  * under the License.
  */
 
-
 import info.aduna.iteration.CloseableIteration;
 
 import java.io.IOException;
@@ -28,6 +27,7 @@ import java.util.Arrays;
 import java.util.Set;
 
 import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.resolver.RdfToRyaConversions;
 import mvm.rya.api.resolver.RyaToRdfConversions;
 import mvm.rya.indexing.GeoIndexer;
 import mvm.rya.indexing.StatementContraints;
@@ -51,209 +51,236 @@ import com.mongodb.MongoCredential;
 import com.mongodb.ServerAddress;
 import com.vividsolutions.jts.geom.Geometry;
 
-public class MongoGeoIndexer extends AbstractMongoIndexer implements GeoIndexer{
-    
-    private static final Logger logger = Logger.getLogger(MongoGeoIndexer.class);
-    
-    private GeoMongoDBStorageStrategy storageStrategy;
-    private MongoClient mongoClient;
-    private DB db;
-    private DBCollection coll;
-    private Set<URI> predicates;
-    private Configuration conf;
-    private boolean isInit = false;
-    private String tableName = "";
-
-    
-    
-    private void init() throws NumberFormatException, UnknownHostException{
-        ServerAddress server = new ServerAddress(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE),
-                Integer.valueOf(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT)));
-        this.conf = conf;
-        if (conf.get(MongoDBRdfConfiguration.MONGO_USER) != null){
-            MongoCredential cred = MongoCredential.createCredential(conf.get(MongoDBRdfConfiguration.MONGO_USER), conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD),
-                    conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME).toCharArray());
-            mongoClient = new MongoClient(server, Arrays.asList(cred));
-        }
-        else {
-            mongoClient = new MongoClient(server);
+import de.flapdoodle.embed.mongo.distribution.Version;
+import de.flapdoodle.embed.mongo.tests.MongodForTestsFactory;
+
+public class MongoGeoIndexer extends AbstractMongoIndexer implements GeoIndexer {
+
+	private static final Logger logger = Logger
+			.getLogger(MongoGeoIndexer.class);
+
+	private GeoMongoDBStorageStrategy storageStrategy;
+	private MongoClient mongoClient;
+	private DB db;
+	private DBCollection coll;
+	private Set<URI> predicates;
+	private Configuration conf;
+	private boolean isInit = false;
+	private String tableName = "";
+
+	private MongodForTestsFactory testsFactory;
+
+	private void init() throws NumberFormatException, IOException{
+        boolean useMongoTest = conf.getBoolean(MongoDBRdfConfiguration.USE_TEST_MONGO, false);
+        if (useMongoTest) {
+            testsFactory = MongodForTestsFactory.with(Version.Main.PRODUCTION);
+            mongoClient = testsFactory.newMongo();
+            int port = mongoClient.getServerAddressList().get(0).getPort();
+            conf.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, Integer.toString(port));
+        } else {
+            ServerAddress server = new ServerAddress(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE),
+                    Integer.valueOf(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT)));
+            if (conf.get(MongoDBRdfConfiguration.MONGO_USER) != null) {
+                MongoCredential cred = MongoCredential.createCredential(
+                        conf.get(MongoDBRdfConfiguration.MONGO_USER),
+                        conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME),
+                        conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD).toCharArray());
+                mongoClient = new MongoClient(server, Arrays.asList(cred));
+            } else {
+                mongoClient = new MongoClient(server);
+            }
         }
         predicates = ConfigUtils.getGeoPredicates(conf);
         tableName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
         db = mongoClient.getDB(tableName);
-        coll = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + "_geo");
+        coll = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya"));
         storageStrategy = new GeoMongoDBStorageStrategy(Double.valueOf(conf.get(MongoDBRdfConfiguration.MONGO_GEO_MAXDISTANCE, "1e-10")));
     }
-    
-
-    @Override
-    public String getTableName() {
-        return tableName;
-    }
-    
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-    
-    //setConf initializes because index is created via reflection
-    @Override
-    public void setConf(Configuration conf) {
-        this.conf = conf;
-        if (!isInit) {
-            try {
-                init();
-                isInit = true;
-            } catch (NumberFormatException e) {
-                logger.warn("Unable to initialize index.  Throwing Runtime Exception. ", e);
-                throw new RuntimeException(e);
-            } catch (UnknownHostException e) {
-                logger.warn("Unable to initialize index.  Throwing Runtime Exception. ", e);
-                throw new RuntimeException(e);
-            }
-        }
-    }
-    
-
-    
-    private void storeStatement(Statement statement) throws IOException {
-        // if this is a valid predicate and a valid geometry
-        boolean isValidPredicate = predicates.isEmpty() || predicates.contains(statement.getPredicate());
-
-        if (isValidPredicate && (statement.getObject() instanceof Literal)) {
-            
-            // add it to the collection
-            try {
-                DBObject obj = storageStrategy.serialize(statement);
-                if (obj != null){
-                    coll.insert(obj);
-                }
-            }
-            catch (com.mongodb.MongoException.DuplicateKey exception){
-                // ignore
-            }
-            catch (com.mongodb.DuplicateKeyException exception){
-                // ignore
-            }
-            catch (Exception ex){
-                // ignore single exceptions
-                ex.printStackTrace();
-            }
-        }
-    }
-    
-    
-    @Override
-    public void storeStatement(RyaStatement statement) throws IOException {
-        storeStatement(RyaToRdfConversions.convertStatement(statement));
-    }
-    
-    
-
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryEquals(
-            Geometry query, StatementContraints contraints) {
-        DBObject queryObj = storageStrategy.getQuery(contraints, query, GeoQueryType.EQUALS);
-        return getIteratorWrapper(queryObj, coll, storageStrategy);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(
-            Geometry query, StatementContraints contraints) {
-        throw new UnsupportedOperationException("Disjoint queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(
-                Geometry query, StatementContraints contraints) {
-        DBObject queryObj = storageStrategy.getQuery(contraints, query, GeoQueryType.INTERSECTS);
-        return getIteratorWrapper(queryObj, coll, storageStrategy);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryTouches(
-            Geometry query, StatementContraints contraints) {
-        throw new UnsupportedOperationException("Touches queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(
-            Geometry query, StatementContraints contraints) {
-        throw new UnsupportedOperationException("Crosses queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryWithin(
-            Geometry query, StatementContraints contraints) {
-        DBObject queryObj = storageStrategy.getQuery(contraints, query, GeoQueryType.WITHIN);
-        return getIteratorWrapper(queryObj, coll, storageStrategy);
-    }
-    
-    
-    private CloseableIteration<Statement, QueryEvaluationException> getIteratorWrapper(final DBObject query, final DBCollection coll, final GeoMongoDBStorageStrategy storageStrategy) {
-
-        return new CloseableIteration<Statement, QueryEvaluationException>() {
-
-            private DBCursor cursor = null;
-
-            private DBCursor getIterator() throws QueryEvaluationException {
-                if (cursor == null){
-                    cursor = coll.find(query);
-                }
-                return cursor;
-            }
-
-            @Override
-            public boolean hasNext() throws QueryEvaluationException {
-                return getIterator().hasNext();
-            }
-
-            @Override
-            public Statement next() throws QueryEvaluationException {
-                DBObject feature = getIterator().next();
-                return storageStrategy.deserializeDBObject(feature);
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException("Remove not implemented");
-            }
-
-            @Override
-            public void close() throws QueryEvaluationException {
-                getIterator().close();
-            }
-        };
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryContains(
-            Geometry query, StatementContraints contraints) {
-        throw new UnsupportedOperationException("Contains queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(
-            Geometry query, StatementContraints contraints) {
-        throw new UnsupportedOperationException("Overlaps queries are not supported in Mongo DB.");
-    }
 
-    @Override
-    public Set<URI> getIndexablePredicates() {
-        return predicates;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void close() throws IOException {
-        mongoClient.close();
-    }
+	@Override
+	public String getTableName() {
+		return tableName;
+	}
+
+	@Override
+	public Configuration getConf() {
+		return conf;
+	}
+
+	// setConf initializes because index is created via reflection
+	@Override
+	public void setConf(Configuration conf) {
+		this.conf = conf;
+		if (!isInit) {
+			try {
+				init();
+				isInit = true;
+			} catch (NumberFormatException e) {
+				logger.warn(
+						"Unable to initialize index.  Throwing Runtime Exception. ",
+						e);
+				throw new RuntimeException(e);
+			} catch (IOException e) {
+				logger.warn(
+						"Unable to initialize index.  Throwing Runtime Exception. ",
+						e);
+				throw new RuntimeException(e);
+			}
+		}
+	}
+
+	private void storeStatement(Statement statement) throws IOException {
+		// if this is a valid predicate and a valid geometry
+		boolean isValidPredicate = predicates.isEmpty()
+				|| predicates.contains(statement.getPredicate());
+
+		if (isValidPredicate && (statement.getObject() instanceof Literal)) {
+
+			// add it to the collection
+			try {
+				DBObject obj = storageStrategy.serialize(statement);
+				if (obj != null) {
+					DBObject query = storageStrategy
+							.getQuery(RdfToRyaConversions
+									.convertStatement(statement));
+					coll.update(query, obj, true, false);
+				}
+			} catch (com.mongodb.MongoException.DuplicateKey exception) {
+				// ignore
+			} catch (com.mongodb.DuplicateKeyException exception) {
+				// ignore
+			} catch (Exception ex) {
+				// ignore single exceptions
+				ex.printStackTrace();
+			}
+		}
+	}
+
+	@Override
+	public void storeStatement(RyaStatement statement) throws IOException {
+		storeStatement(RyaToRdfConversions.convertStatement(statement));
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryEquals(
+			Geometry query, StatementContraints contraints) {
+		DBObject queryObj = storageStrategy.getQuery(contraints, query,
+				GeoQueryType.EQUALS);
+		return getIteratorWrapper(queryObj, coll, storageStrategy);
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(
+			Geometry query, StatementContraints contraints) {
+		throw new UnsupportedOperationException(
+				"Disjoint queries are not supported in Mongo DB.");
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(
+			Geometry query, StatementContraints contraints) {
+		DBObject queryObj = storageStrategy.getQuery(contraints, query,
+				GeoQueryType.INTERSECTS);
+		return getIteratorWrapper(queryObj, coll, storageStrategy);
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryTouches(
+			Geometry query, StatementContraints contraints) {
+		throw new UnsupportedOperationException(
+				"Touches queries are not supported in Mongo DB.");
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(
+			Geometry query, StatementContraints contraints) {
+		throw new UnsupportedOperationException(
+				"Crosses queries are not supported in Mongo DB.");
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryWithin(
+			Geometry query, StatementContraints contraints) {
+		DBObject queryObj = storageStrategy.getQuery(contraints, query,
+				GeoQueryType.WITHIN);
+		return getIteratorWrapper(queryObj, coll, storageStrategy);
+	}
+
+	private CloseableIteration<Statement, QueryEvaluationException> getIteratorWrapper(
+			final DBObject query, final DBCollection coll,
+			final GeoMongoDBStorageStrategy storageStrategy) {
+
+		return new CloseableIteration<Statement, QueryEvaluationException>() {
+
+			private DBCursor cursor = null;
+
+			private DBCursor getIterator() throws QueryEvaluationException {
+				if (cursor == null) {
+					cursor = coll.find(query);
+				}
+				return cursor;
+			}
+
+			@Override
+			public boolean hasNext() throws QueryEvaluationException {
+				return getIterator().hasNext();
+			}
+
+			@Override
+			public Statement next() throws QueryEvaluationException {
+				DBObject feature = getIterator().next();
+				RyaStatement statement = storageStrategy
+						.deserializeDBObject(feature);
+				return RyaToRdfConversions.convertStatement(statement);
+			}
+
+			@Override
+			public void remove() {
+				throw new UnsupportedOperationException(
+						"Remove not implemented");
+			}
+
+			@Override
+			public void close() throws QueryEvaluationException {
+				getIterator().close();
+			}
+		};
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryContains(
+			Geometry query, StatementContraints contraints) {
+		throw new UnsupportedOperationException(
+				"Contains queries are not supported in Mongo DB.");
+	}
+
+	@Override
+	public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(
+			Geometry query, StatementContraints contraints) {
+		throw new UnsupportedOperationException(
+				"Overlaps queries are not supported in Mongo DB.");
+	}
+
+	@Override
+	public Set<URI> getIndexablePredicates() {
+		return predicates;
+	}
+
+	@Override
+	public void flush() throws IOException {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public void close() throws IOException {
+		mongoClient.close();
+	}
+
+	@Override
+	public void deleteStatement(RyaStatement stmt) throws IOException {
+	   DBObject obj = storageStrategy.getQuery(stmt);
+	   coll.remove(obj);
+	}
 
-    
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerSfTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerSfTest.java
new file mode 100644
index 0000000..7e1eaec
--- /dev/null
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerSfTest.java
@@ -0,0 +1,306 @@
+package mvm.rya.indexing.mongo;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+
+import info.aduna.iteration.CloseableIteration;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import mvm.rya.api.RdfCloudTripleStoreConfiguration;
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.resolver.RdfToRyaConversions;
+import mvm.rya.api.resolver.RyaToRdfConversions;
+import mvm.rya.indexing.StatementContraints;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.accumulo.geo.GeoConstants;
+import mvm.rya.indexing.accumulo.geo.GeoMesaGeoIndexer;
+import mvm.rya.indexing.mongodb.MongoGeoIndexer;
+import mvm.rya.mongodb.MongoDBRdfConfiguration;
+
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+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 com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.LineString;
+import com.vividsolutions.jts.geom.LinearRing;
+import com.vividsolutions.jts.geom.Point;
+import com.vividsolutions.jts.geom.Polygon;
+import com.vividsolutions.jts.geom.PrecisionModel;
+import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
+
+/**
+ * Tests all of the "simple functions" of the geoindexer.
+ */
+public class MongoGeoIndexerSfTest {
+    private static Configuration conf;
+    private static GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
+    private static MongoGeoIndexer g;
+
+    private static final StatementContraints EMPTY_CONSTRAINTS = new StatementContraints();
+
+    // Here is the landscape:
+    /**
+     * <pre>
+     * 	 +---+---+---+---+---+---+---+
+     * 	 |        F          |       |
+     * 	 +  A    +           +   C   +
+     * 	 |                   |       |
+     * 	 +---+---+    E      +---+---+
+     * 	 |       |   /       |
+     * 	 +   B   +  /+---+---+
+     * 	 |       | / |       |
+     * 	 +---+---+/--+---+---+
+     * 	         /   |     D |
+     * 	        /    +---+---+
+     * </pre>
+     **/
+
+    private static final Polygon A = poly(bbox(0, 1, 4, 5));
+    private static final Polygon B = poly(bbox(0, 1, 2, 3));
+    private static final Polygon C = poly(bbox(4, 3, 6, 5));
+    private static final Polygon D = poly(bbox(3, 0, 5, 2));
+
+    private static final Point F = point(2, 4);
+
+    private static final LineString E = line(2, 0, 3, 3);
+
+    private static final Map<Geometry, String> names = Maps.newHashMap();
+    static {
+        names.put(A, "A");
+        names.put(B, "B");
+        names.put(C, "C");
+        names.put(D, "D");
+        names.put(E, "E");
+        names.put(F, "F");
+    }
+
+    @Before
+    public void before() throws Exception {
+        System.out.println(UUID.randomUUID().toString());
+        conf = new Configuration();
+        conf.set(ConfigUtils.USE_MONGO, "true");
+        conf.set(MongoDBRdfConfiguration.USE_TEST_MONGO, "true");
+        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
+        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        conf.set(ConfigUtils.USE_GEO, "true");
+        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
+
+        g = new MongoGeoIndexer();
+        g.setConf(conf);
+        g.storeStatement(statement(A));
+        g.storeStatement(statement(B));
+        g.storeStatement(statement(C));
+        g.storeStatement(statement(D));
+        g.storeStatement(statement(F));
+        g.storeStatement(statement(E));
+    }
+
+    private static RyaStatement statement(Geometry geo) {
+        ValueFactory vf = new ValueFactoryImpl();
+        Resource subject = vf.createURI("uri:" + names.get(geo));
+        URI predicate = GeoConstants.GEO_AS_WKT;
+        Value object = vf.createLiteral(geo.toString(), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+
+    }
+
+    private static Point point(double x, double y) {
+        return gf.createPoint(new Coordinate(x, y));
+    }
+
+    private static LineString line(double x1, double y1, double x2, double y2) {
+        return new LineString(new PackedCoordinateSequence.Double(new double[] { x1, y1, x2, y2 }, 2), gf);
+    }
+
+    private static Polygon poly(double[] arr) {
+        LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(arr, 2));
+        Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+        return p1;
+    }
+
+    private static double[] bbox(double x1, double y1, double x2, double y2) {
+        return new double[] { x1, y1, x1, y2, x2, y2, x2, y1, x1, y1 };
+    }
+
+    public void compare(CloseableIteration<Statement, ?> actual, Geometry... expected) throws Exception {
+        Set<Statement> expectedSet = Sets.newHashSet();
+        for (Geometry geo : expected) {
+            expectedSet.add(RyaToRdfConversions.convertStatement(statement(geo)));
+        }
+
+        Assert.assertEquals(expectedSet, getSet(actual));
+    }
+
+    private static <X> Set<X> getSet(CloseableIteration<X, ?> iter) throws Exception {
+        Set<X> set = new HashSet<X>();
+        while (iter.hasNext()) {
+            set.add(iter.next());
+        }
+        return set;
+    }
+
+    private static Geometry[] EMPTY_RESULTS = {};
+
+    @Test
+    public void testEquals() throws Exception {
+        // point
+        compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
+        compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+        // line
+        compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
+        compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+        // poly
+        compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
+        compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+    }
+
+//    @Test
+//    public void testDisjoint() throws Exception {
+//        // point
+//        compare(g.queryDisjoint(F, EMPTY_CONSTRAINTS), B, C, D, E);
+//
+//        // line
+//        compare(g.queryDisjoint(E, EMPTY_CONSTRAINTS), B, C, D, F);
+//
+//        // poly
+//        compare(g.queryDisjoint(A, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//        compare(g.queryDisjoint(B, EMPTY_CONSTRAINTS), C, D, F, E);
+//    }
+
+    @Test
+    public void testIntersectsPoint() throws Exception {
+        // This seems like a bug
+        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), A, F);
+        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+    }
+
+    @Test
+    public void testIntersectsLine() throws Exception {
+        // This seems like a bug
+        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), A, E);
+        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+    }
+
+//    @Test
+//    public void testIntersectsPoly() throws Exception {
+//        compare(g.queryIntersects(A, EMPTY_CONSTRAINTS), A, B, C, D, F, E);
+//    }
+
+//    @Test
+//    public void testTouchesPoint() throws Exception {
+//        compare(g.queryTouches(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+//
+//    @Test
+//    public void testTouchesLine() throws Exception {
+//        compare(g.queryTouches(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testTouchesPoly() throws Exception {
+//        compare(g.queryTouches(A, EMPTY_CONSTRAINTS), C);
+//    }
+
+//    @Test
+//    public void testCrossesPoint() throws Exception {
+//        compare(g.queryCrosses(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+    @Test
+    public void testCrossesLine() throws Exception {
+        // compare(g.queryCrosses(E, EMPTY_CONSTRAINTS), A);
+    }
+
+//    @Test
+//    public void testCrossesPoly() throws Exception {
+//        compare(g.queryCrosses(A, EMPTY_CONSTRAINTS), E);
+//    }
+
+//    @Test
+//    public void testWithin() throws Exception {
+//        // point
+//  //      compare(g.queryWithin(F, EMPTY_CONSTRAINTS), F);
+//
+//        // line
+////        compare(g.queryWithin(E, EMPTY_CONSTRAINTS), E);
+//
+//        // poly
+//        compare(g.queryWithin(A, EMPTY_CONSTRAINTS), A, B, F);
+//    }
+
+//    @Test
+//    public void testContainsPoint() throws Exception {
+//        compare(g.queryContains(F, EMPTY_CONSTRAINTS), A, F);
+//    }
+
+    @Test
+    public void testContainsLine() throws Exception {
+        // compare(g.queryContains(E, EMPTY_CONSTRAINTS), E);
+    }
+
+//    @Test
+//    public void testContainsPoly() throws Exception {
+//        compare(g.queryContains(A, EMPTY_CONSTRAINTS), A);
+//        compare(g.queryContains(B, EMPTY_CONSTRAINTS), A, B);
+//    }
+
+    @Test
+    public void testOverlapsPoint() throws Exception {
+        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), F);
+        // You cannot have overlapping points
+        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+    }
+
+    @Test
+    public void testOverlapsLine() throws Exception {
+        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), A, E);
+        // You cannot have overlapping lines
+        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+    }
+
+//    @Test
+//    public void testOverlapsPoly() throws Exception {
+//        compare(g.queryOverlaps(A, EMPTY_CONSTRAINTS), D);
+//    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7743a42a/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerTest.java
new file mode 100644
index 0000000..4075b29
--- /dev/null
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/mongo/MongoGeoIndexerTest.java
@@ -0,0 +1,390 @@
+package mvm.rya.indexing.mongo;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+
+import static mvm.rya.api.resolver.RdfToRyaConversions.convertStatement;
+import info.aduna.iteration.CloseableIteration;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import mvm.rya.api.RdfCloudTripleStoreConfiguration;
+import mvm.rya.indexing.StatementContraints;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.accumulo.geo.GeoConstants;
+import mvm.rya.indexing.mongodb.MongoGeoIndexer;
+import mvm.rya.mongodb.MongoDBRdfConfiguration;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+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.ContextStatementImpl;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.Sets;
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.LinearRing;
+import com.vividsolutions.jts.geom.Point;
+import com.vividsolutions.jts.geom.Polygon;
+import com.vividsolutions.jts.geom.PrecisionModel;
+import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
+
+public class MongoGeoIndexerTest {
+
+    private static final StatementContraints EMPTY_CONSTRAINTS = new StatementContraints();
+
+    Configuration conf;
+    GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
+
+    @Before
+    public void before() throws Exception {
+        conf = new Configuration();
+        conf.set(ConfigUtils.USE_MONGO, "true");
+        conf.set(MongoDBRdfConfiguration.USE_TEST_MONGO, "true");
+        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
+        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        conf.set(ConfigUtils.USE_GEO, "true");
+        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
+    }
+
+    @Test
+    public void testRestrictPredicatesSearch() throws Exception {
+        conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+
+            Point point = gf.createPoint(new Coordinate(10, 10));
+            Value pointValue = vf.createLiteral("Point(10 10)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            URI invalidPredicate = GeoConstants.GEO_AS_WKT;
+
+            // These should not be stored because they are not in the predicate list
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), invalidPredicate, pointValue)));
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), invalidPredicate, pointValue)));
+
+            URI pred1 = vf.createURI("pred:1");
+            URI pred2 = vf.createURI("pred:2");
+
+            // These should be stored because they are in the predicate list
+            Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1, pointValue);
+            Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2, pointValue);
+            f.storeStatement(convertStatement(s3));
+            f.storeStatement(convertStatement(s4));
+
+            // This should not be stored because the object is not valid wkt
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1, vf.createLiteral("soint(10 10)"))));
+
+            // This should not be stored because the object is not a literal
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj6"), pred1, vf.createURI("p:Point(10 10)"))));
+
+            f.flush();
+
+            Set<Statement> actual = getSet(f.queryEquals(point, EMPTY_CONSTRAINTS));
+            Assert.assertEquals(2, actual.size());
+            Assert.assertTrue(actual.contains(s3));
+            Assert.assertTrue(actual.contains(s4));
+        }
+    }
+
+    private static <X> Set<X> getSet(CloseableIteration<X, ?> iter) throws Exception {
+        Set<X> set = new HashSet<X>();
+        while (iter.hasNext()) {
+            set.add(iter.next());
+        }
+        return set;
+    }
+
+    @Test
+    public void testPrimeMeridianSearch() throws Exception {
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] ONE = { 1, 1, -1, 1, -1, -1, 1, -1, 1, 1 };
+            double[] TWO = { 2, 2, -2, 2, -2, -2, 2, -2, 2, 2 };
+            double[] THREE = { 3, 3, -3, 3, -3, -3, 3, -3, 3, 3 };
+
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
+            LinearRing r2 = gf.createLinearRing(new PackedCoordinateSequence.Double(TWO, 2));
+            LinearRing r3 = gf.createLinearRing(new PackedCoordinateSequence.Double(THREE, 2));
+
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            Polygon p2 = gf.createPolygon(r2, new LinearRing[] {});
+            Polygon p3 = gf.createPolygon(r3, new LinearRing[] {});
+
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p2, EMPTY_CONSTRAINTS)));
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p3, EMPTY_CONSTRAINTS)));
+
+            // Test a ring with a hole in it
+            Polygon p3m2 = gf.createPolygon(r3, new LinearRing[] { r2 });
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p3m2, EMPTY_CONSTRAINTS)));
+
+            // test a ring outside the point
+            double[] OUT = { 3, 3, 1, 3, 1, 1, 3, 1, 3, 3 };
+            LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
+            Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDcSearch() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+
+            // test a ring outside the point
+            double[] OUT = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
+            LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
+            Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDeleteSearch() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            f.deleteStatement(convertStatement(statement));
+
+            // test a ring that the point would be inside of if not deleted
+            double[] in = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(in, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+
+            // test a ring that the point would be outside of if not deleted
+            double[] out = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
+            LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(out, 2));
+            Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+
+            // test a ring for the whole world and make sure the point is gone
+            // Geomesa is a little sensitive around lon 180, so we only go to 179
+            double[] world = { -180, 90, 179, 90, 179, -90, -180, -90, -180, 90 };
+            LinearRing rWorld = gf.createLinearRing(new PackedCoordinateSequence.Double(world, 2));
+            Polygon pWorld = gf.createPolygon(rWorld, new LinearRing[] {});
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pWorld, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithContext() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct context
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementContraints().setContext(context))));
+
+            // query with wrong context
+            Assert.assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementContraints().setContext(vf.createURI("foo:context2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithSubject() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct subject
+            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementContraints().setSubject(subject))));
+
+            // query with wrong subject
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementContraints().setSubject(vf.createURI("foo:subj2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithSubjectAndContext() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct context subject
+            Assert.assertEquals(Sets.newHashSet(statement),
+                    getSet(f.queryWithin(p1, new StatementContraints().setContext(context).setSubject(subject))));
+
+            // query with wrong context
+            Assert.assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementContraints().setContext(vf.createURI("foo:context2")))));
+
+            // query with wrong subject
+            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementContraints().setSubject(vf.createURI("foo:subj2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithPredicate() throws Exception {
+        // test a ring around dc
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource subject = vf.createURI("foo:subj");
+            URI predicate = GeoConstants.GEO_AS_WKT;
+            Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Resource context = vf.createURI("foo:context");
+
+            Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct Predicate
+            Assert.assertEquals(Sets.newHashSet(statement),
+                    getSet(f.queryWithin(p1, new StatementContraints().setPredicates(Collections.singleton(predicate)))));
+
+            // query with wrong predicate
+            Assert.assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementContraints().setPredicates(Collections.singleton(vf.createURI("other:pred"))))));
+        }
+    }
+
+    // @Test
+    public void testAntiMeridianSearch() throws Exception {
+        // verify that a search works if the bounding box crosses the anti meridian
+        try (MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+
+            ValueFactory vf = new ValueFactoryImpl();
+            Resource context = vf.createURI("foo:context");
+
+            Resource subjectEast = vf.createURI("foo:subj:east");
+            URI predicateEast = GeoConstants.GEO_AS_WKT;
+            Value objectEast = vf.createLiteral("Point(179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Statement statementEast = new ContextStatementImpl(subjectEast, predicateEast, objectEast, context);
+            f.storeStatement(convertStatement(statementEast));
+
+            Resource subjectWest = vf.createURI("foo:subj:west");
+            URI predicateWest = GeoConstants.GEO_AS_WKT;
+            Value objectWest = vf.createLiteral("Point(-179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            Statement statementWest = new ContextStatementImpl(subjectWest, predicateWest, objectWest, context);
+            f.storeStatement(convertStatement(statementWest));
+
+            f.flush();
+
+            double[] ONE = { 178.1, 1, -178, 1, -178, -1, 178.1, -1, 178.1, 1 };
+
+            LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
+
+            Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            Assert.assertEquals(Sets.newHashSet(statementEast, statementWest), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+        }
+    }
+}