You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by dl...@apache.org on 2017/08/30 20:31:42 UTC

[04/14] incubator-rya git commit: RYA-324, RYA-272 Geo refactoring and examples closes #182

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e76b8d7/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
deleted file mode 100644
index 34df399..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
+++ /dev/null
@@ -1,226 +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.geotemporal.mongo;
-
-import static com.google.common.base.Preconditions.checkState;
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Logger;
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.api.resolver.RyaToRdfConversions;
-import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.indexing.TemporalInstant;
-import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.TemporalInterval;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.accumulo.geo.GeoParseUtils;
-import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer;
-import org.apache.rya.indexing.geotemporal.model.Event;
-import org.apache.rya.indexing.geotemporal.storage.EventStorage;
-import org.apache.rya.indexing.mongodb.AbstractMongoIndexer;
-import org.apache.rya.indexing.mongodb.IndexingException;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.joda.time.DateTime;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.io.ParseException;
-
-/**
- * Indexer that stores 2 separate statements as one 'Event' entity.
- * <p>
- * The statements are required to have the same subject, one must be
- * a Geo based statement and the other a temporal based statement.
- * <p>
- * This indexer is later used when querying for geo/temporal statements
- * in the format of:
- * <pre>
- * {@code
- * QUERY PARAMS
- *   ?SomeSubject geo:predicate ?Location
- *   ?SomeSubject time:predicate ?Time
- *   Filter(?Location, geoFunction())
- *   Filter(?Time, temporalFunction())
- * }
- *
- * The number of filters is not strict, but there must be at least one
- * query pattern for geo and one for temporal as well as at least one
- * filter for each type.
- */
-public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMongoDBStorageStrategy> implements GeoTemporalIndexer {
-    private static final Logger LOG = Logger.getLogger(MongoGeoTemporalIndexer.class);
-    public static final String GEO_TEMPORAL_COLLECTION = "geo_temporal";
-
-    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
-    private final AtomicReference<EventStorage> events = new AtomicReference<>();
-
-    @Override
-    public void init() {
-        initCore();
-        predicates = ConfigUtils.getGeoPredicates(conf);
-        predicates.addAll(ConfigUtils.getTemporalPredicates(conf));
-        storageStrategy = new GeoTemporalMongoDBStorageStrategy();
-    }
-
-    @Override
-    public void setConf(final Configuration conf) {
-        requireNonNull(conf);
-        events.set(null);
-        events.set(getEventStorage(conf));
-        super.conf = conf;
-        configuration.set(new MongoDBRdfConfiguration(conf));
-    }
-
-    @Override
-    public void storeStatement(final RyaStatement ryaStatement) throws IOException {
-        requireNonNull(ryaStatement);
-
-        try {
-            updateEvent(ryaStatement.getSubject(), ryaStatement);
-        } catch (IndexingException | ParseException e) {
-            throw new IOException("Failed to update the Entity index.", e);
-        }
-    }
-
-    @Override
-    public void deleteStatement(final RyaStatement statement) throws IOException {
-        requireNonNull(statement);
-        final RyaURI subject = statement.getSubject();
-        try {
-            final EventStorage eventStore = events.get();
-            checkState(events != null, "Must set this indexers configuration before storing statements.");
-
-            new EventUpdater(eventStore).update(subject, old -> {
-                final Event.Builder updated;
-                if(!old.isPresent()) {
-                    return Optional.empty();
-                } else {
-                    updated = Event.builder(old.get());
-                }
-
-                final Event currentEvent = updated.build();
-                final URI pred = statement.getObject().getDataType();
-                if((pred.equals(GeoConstants.GEO_AS_WKT) || pred.equals(GeoConstants.GEO_AS_GML) ||
-                   pred.equals(GeoConstants.XMLSCHEMA_OGC_WKT) || pred.equals(GeoConstants.XMLSCHEMA_OGC_GML))
-                   && currentEvent.getGeometry().isPresent()) {
-                    //is geo and needs to be removed.
-                    try {
-                        if(currentEvent.getGeometry().get().equals(GeoParseUtils.getGeometry(RyaToRdfConversions.convertStatement(statement)))) {
-                            updated.setGeometry(null);
-                        }
-                    } catch (final Exception e) {
-                        LOG.debug("Unable to parse the stored geometry.");
-                    }
-                } else {
-                    //is time
-                    final String dateTime = statement.getObject().getData();
-                    final Matcher matcher = TemporalInstantRfc3339.PATTERN.matcher(dateTime);
-                    if (matcher.find()) {
-                        final TemporalInterval interval = TemporalInstantRfc3339.parseInterval(dateTime);
-                        if(currentEvent.getInterval().get().equals(interval)) {
-                            updated.setTemporalInterval(null);
-                        }
-                    } else {
-                        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.parse(dateTime));
-                        if(currentEvent.getInstant().get().equals(instant)) {
-                            updated.setTemporalInstant(null);
-                        }
-                    }
-                }
-                return Optional.of(updated.build());
-            });
-        } catch (final IndexingException e) {
-            throw new IOException("Failed to update the Entity index.", e);
-        }
-    }
-
-    private void updateEvent(final RyaURI subject, final RyaStatement statement) throws IndexingException, ParseException {
-        final EventStorage eventStore = events.get();
-        checkState(events != null, "Must set this indexers configuration before storing statements.");
-
-        new EventUpdater(eventStore).update(subject, old -> {
-            final Event.Builder updated;
-            if(!old.isPresent()) {
-                updated = Event.builder()
-                    .setSubject(subject);
-            } else {
-                updated = Event.builder(old.get());
-            }
-
-            final URI pred = statement.getObject().getDataType();
-            if(pred.equals(GeoConstants.GEO_AS_WKT) || pred.equals(GeoConstants.GEO_AS_GML) ||
-               pred.equals(GeoConstants.XMLSCHEMA_OGC_WKT) || pred.equals(GeoConstants.XMLSCHEMA_OGC_GML)) {
-                //is geo
-                try {
-                    final Statement geoStatement = RyaToRdfConversions.convertStatement(statement);
-                    final Geometry geometry = GeoParseUtils.getGeometry(geoStatement);
-                    updated.setGeometry(geometry);
-                } catch (final ParseException e) {
-                    LOG.error(e.getMessage(), e);
-                }
-            } else {
-                //is time
-                final String dateTime = statement.getObject().getData();
-                final Matcher matcher = TemporalInstantRfc3339.PATTERN.matcher(dateTime);
-                if (matcher.find()) {
-                    final TemporalInterval interval = TemporalInstantRfc3339.parseInterval(dateTime);
-                    updated.setTemporalInterval(interval);
-                } else {
-                    final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.parse(dateTime));
-                    updated.setTemporalInstant(instant);
-                }
-            }
-            return Optional.of(updated.build());
-        });
-    }
-
-    @Override
-    public String getCollectionName() {
-        return ConfigUtils.getTablePrefix(conf)  + GEO_TEMPORAL_COLLECTION;
-    }
-
-    @Override
-    public EventStorage getEventStorage(final Configuration conf) {
-        requireNonNull(conf);
-
-        if(events.get() != null) {
-            return events.get();
-        }
-
-
-        final MongoDBRdfConfiguration mongoConf = new MongoDBRdfConfiguration(conf);
-        mongoClient = mongoConf.getMongoClient();
-        configuration.set(mongoConf);
-        if (mongoClient == null) {
-            mongoClient = MongoConnectorFactory.getMongoClient(conf);
-        }
-        final String ryaInstanceName = mongoConf.getMongoDBName();
-        events.set(new MongoEventStorage(mongoClient, ryaInstanceName));
-        return events.get();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e76b8d7/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/storage/EventStorage.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/storage/EventStorage.java b/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/storage/EventStorage.java
deleted file mode 100644
index 47c18a0..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/geotemporal/storage/EventStorage.java
+++ /dev/null
@@ -1,130 +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.geotemporal.storage;
-
-import java.util.Collection;
-import java.util.Optional;
-
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.indexing.IndexingExpr;
-import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer;
-import org.apache.rya.indexing.geotemporal.model.Event;
-import org.apache.rya.indexing.mongodb.update.RyaObjectStorage;
-
-public interface EventStorage extends RyaObjectStorage<Event> {
-    /**
-     * Search for {@link Event}s from the storage by its subject.
-     * Will query based on present parameters.
-     *
-     * @param subject - The subject key to find events.
-     * @param geoFilters - The geo filters to find Events.
-     * @param temporalFilters - The temporal filters to find Events.
-     * @return The {@link Event}, if one exists for the subject.
-     * @throws ObjectStorageException A problem occurred while fetching the Entity from the storage.
-     */
-    public Collection<Event> search(final Optional<RyaURI> subject, Optional<Collection<IndexingExpr>> geoFilters, Optional<Collection<IndexingExpr>> temporalFilters) throws ObjectStorageException;
-
-    /**
-     * Indicates a problem while interacting with an {@link EventStorage}.
-     */
-    public static class EventStorageException extends ObjectStorageException {
-        private static final long serialVersionUID = 1L;
-
-        /**
-         * Constructs a new exception with the specified detail message.  The
-         * cause is not initialized, and may subsequently be initialized by
-         * a call to {@link #initCause}.
-         *
-         * @param   message   the detail message. The detail message is saved for
-         *          later retrieval by the {@link #getMessage()} method.
-         */
-        public EventStorageException(final String message) {
-            super(message);
-        }
-
-        /**
-         * Constructs a new exception with the specified detail message and
-         * cause.  <p>Note that the detail message associated with
-         * {@code cause} is <i>not</i> automatically incorporated in
-         * this exception's detail message.
-         *
-         * @param  message the detail message (which is saved for later retrieval
-         *         by the {@link #getMessage()} method).
-         * @param  cause the cause (which is saved for later retrieval by the
-         *         {@link #getCause()} method).  (A <tt>null</tt> value is
-         *         permitted, and indicates that the cause is nonexistent or
-         *         unknown.)
-         */
-        public EventStorageException(final String message, final Throwable cause) {
-            super(message, cause);
-        }
-    }
-
-    /**
-     * An {@link Event} could not be created because one already exists for the Subject.
-     */
-    public static class EventAlreadyExistsException extends EventStorageException {
-        private static final long serialVersionUID = 1L;
-
-        public EventAlreadyExistsException(final String message) {
-            super(message);
-        }
-
-        public EventAlreadyExistsException(final String message, final Throwable cause) {
-            super(message, cause);
-        }
-    }
-
-    /**
-     * An {@link TypedEvent} could not be updated because the old state does not
-     * match the current state.
-     */
-    public static class StaleUpdateException extends EventStorageException {
-        private static final long serialVersionUID = 1L;
-
-        public StaleUpdateException(final String message) {
-            super(message);
-        }
-
-        public StaleUpdateException(final String message, final Throwable cause) {
-            super(message, cause);
-        }
-    }
-
-    /**
-     *  A {@link EventFilter} is a translation from an {@link IndexingExpr}
-     *  to a format the {@link GeoTemporalIndexer} can use to easily determine which
-     *  filter function is being used.
-     *
-     *   @param T - The type of
-     */
-    interface EventFilter<T> {
-        /**
-         * Gets the translated query friendly form of the filter.
-         */
-        public T getQueryObject();
-    }
-
-    /**
-     * Factory for getting the {@link EventFilter} from an {@link IndexingExpr}.
-     */
-    interface EventFilterFactory<T> {
-        public EventFilter<T> getSearchFunction(final IndexingExpr filter);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e76b8d7/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
deleted file mode 100644
index 634359f..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/GeoMongoDBStorageStrategy.java
+++ /dev/null
@@ -1,247 +0,0 @@
-package org.apache.rya.indexing.mongodb.geo;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.log4j.Logger;
-import org.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;
-
-public class GeoMongoDBStorageStrategy extends IndexingMongoDBStorageStrategy {
-    private static final Logger LOG = Logger.getLogger(GeoMongoDBStorageStrategy.class);
-
-    private static final String GEO = "location";
-    public enum GeoQueryType {
-        INTERSECTS {
-            @Override
-            public String getKeyword() {
-                return "$geoIntersects";
-            }
-        }, WITHIN {
-            @Override
-            public String getKeyword() {
-                return "$geoWithin";
-            }
-        }, EQUALS {
-            @Override
-            public String getKeyword() {
-                return "$near";
-            }
-        }, NEAR {
-            @Override
-            public String getKeyword() {
-                return "$near";
-            }
-        };
-
-        public abstract String getKeyword();
-    }
-
-    public static class GeoQuery {
-        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;
-
-    public GeoMongoDBStorageStrategy(final Double maxDistance) {
-        this.maxDistance = maxDistance;
-    }
-
-    @Override
-    public void createIndices(final DBCollection coll){
-        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.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)){
-            if(geo.getNumPoints() == 1) {
-                final List circle = new ArrayList();
-                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, 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 {
-            final BasicDBObject geoDoc = new BasicDBObject("$geometry", getCorrespondingPoints(geo));
-            query = new BasicDBObject(GEO, new BasicDBObject(queryType.getKeyword(), geoDoc));
-        }
-
-        return query;
-    }
-
-    @Override
-    public DBObject serialize(final RyaStatement ryaStatement) {
-        // if the object is wkt, then try to index it
-        // write the statement data to the fields
-        try {
-            final Statement statement = RyaToRdfConversions.convertStatement(ryaStatement);
-            final Geometry geo = (new WKTReader()).read(GeoParseUtils.getWellKnownText(statement));
-            if(geo == null) {
-                LOG.error("Failed to parse geo statement: " + statement.toString());
-                return null;
-            }
-            final BasicDBObject base = (BasicDBObject) super.serialize(ryaStatement);
-            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);
-            return null;
-        }
-    }
-
-    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/9e76b8d7/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
deleted file mode 100644
index 2abee76..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoIndexer.java
+++ /dev/null
@@ -1,154 +0,0 @@
-package org.apache.rya.indexing.mongodb.geo;
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import static 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;
-import org.openrdf.model.Statement;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.mongodb.DBObject;
-import com.vividsolutions.jts.geom.Geometry;
-
-import info.aduna.iteration.CloseableIteration;
-
-public class MongoGeoIndexer extends AbstractMongoIndexer<GeoMongoDBStorageStrategy> implements GeoIndexer {
-    private static final String COLLECTION_SUFFIX = "geo";
-    private static final Logger logger = Logger.getLogger(MongoGeoIndexer.class);
-
-    @Override
-	public void init() {
-        initCore();
-        predicates = ConfigUtils.getGeoPredicates(conf);
-        if(predicates.size() == 0) {
-            logger.debug("No predicates specified for geo indexing.  During insertion, all statements will be attempted to be indexed into the geo indexer.");
-        }
-        storageStrategy = new GeoMongoDBStorageStrategy(Double.valueOf(conf.get(MongoDBRdfConfiguration.MONGO_GEO_MAXDISTANCE, "1e-10")));
-        storageStrategy.createIndices(collection);
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryEquals(
-            final Geometry query, final StatementConstraints constraints) {
-        try {
-            final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(EQUALS, query));
-            return withConstraints(constraints, queryObj);
-        } catch (final MalformedQueryException e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryDisjoint(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Disjoint queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryIntersects(
-            final Geometry query, final StatementConstraints constraints) {
-        try {
-            final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(INTERSECTS, query));
-            return withConstraints(constraints, queryObj);
-        } catch (final MalformedQueryException e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryTouches(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Touches queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryCrosses(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Crosses queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryWithin(
-            final Geometry query, final StatementConstraints constraints) {
-        try {
-            final DBObject queryObj = storageStrategy.getQuery(new GeoQuery(WITHIN, query));
-            return withConstraints(constraints, queryObj);
-        } catch (final MalformedQueryException e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    @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(
-                "Contains queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public CloseableIteration<Statement, QueryEvaluationException> queryOverlaps(
-            final Geometry query, final StatementConstraints constraints) {
-        throw new UnsupportedOperationException(
-                "Overlaps queries are not supported in Mongo DB.");
-    }
-
-    @Override
-    public String getCollectionName() {
-        return ConfigUtils.getTablePrefix(conf)  + COLLECTION_SUFFIX;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e76b8d7/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 c564d02..0000000
--- a/extras/rya.geoindexing/src/main/java/org/apache/rya/indexing/mongodb/geo/MongoGeoTupleSet.java
+++ /dev/null
@@ -1,361 +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.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/9e76b8d7/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/GeoIndexingTestUtils.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/GeoIndexingTestUtils.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/GeoIndexingTestUtils.java
deleted file mode 100644
index b0c636d..0000000
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/GeoIndexingTestUtils.java
+++ /dev/null
@@ -1,51 +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;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Utility methods to help test geo indexing methods.
- */
-public final class GeoIndexingTestUtils {
-    /**
-     * Private constructor to prevent instantiation.
-     */
-    private GeoIndexingTestUtils () {
-    }
-
-    /**
-     * Generates a set of items from the specified iterator.
-     * @param iter a {@link CloseableIteration}.
-     * @return the {@link Set} of items from the iterator or an empty set if
-     * none were found.
-     * @throws Exception
-     */
-    public static <X> Set<X> getSet(final CloseableIteration<X, ?> iter) throws Exception {
-        final Set<X> set = new HashSet<X>();
-        while (iter.hasNext()) {
-            final X item = iter.next();
-            set.add(item);
-        }
-        return set;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e76b8d7/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/accumulo/geo/GeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/accumulo/geo/GeoIndexerSfTest.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/accumulo/geo/GeoIndexerSfTest.java
deleted file mode 100644
index e61ef35..0000000
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/accumulo/geo/GeoIndexerSfTest.java
+++ /dev/null
@@ -1,521 +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.accumulo.geo;
-
-import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-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.GeoIndexerType;
-import org.apache.rya.indexing.OptionalConfigUtils;
-import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.geotools.geometry.jts.Geometries;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-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.URIImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import com.google.common.collect.ImmutableMap;
-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;
-import com.vividsolutions.jts.io.ParseException;
-import com.vividsolutions.jts.io.gml2.GMLWriter;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Tests all of the "simple functions" of the geoindexer specific to GML.
- * Parameterized so that each test is run for WKT and for GML.
- */
-@RunWith(value = Parameterized.class)
-public class GeoIndexerSfTest {
-    private static AccumuloRdfConfiguration conf;
-    private static GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
-    private static GeoMesaGeoIndexer g;
-
-    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-
-    // Here is the landscape:
-    /**
-     * <pre>
-     *   2---+---+---+---+---+---+
-     *   |        F      |G      |
-     *   1  A    o(-1,1) o   C   |
-     *   |               |       |
-     *   0---+---+       +---+---+(3,0)
-     *   |       |    E  |
-     *  -1   B   +   .---+---+
-     *   |       |  /|   |   |
-     *  -2---+---+-/-+---+   +
-     *   ^        /  |     D |
-     *  -3  -2  -1   0---1---2   3   4
-     * </pre>
-     **/
-    private static final Polygon A = poly(bbox(-3, -2, 1, 2));
-    private static final Polygon B = poly(bbox(-3, -2, -1, 0));
-    private static final Polygon C = poly(bbox(1, 0, 3, 2));
-    private static final Polygon D = poly(bbox(0, -3, 2, -1));
-
-    private static final Point F = point(-1, 1);
-    private static final Point G = point(1, 1);
-
-    private static final LineString E = line(-1, -3, 0, -1);
-
-    private static final Map<Geometry, String> NAMES = ImmutableMap.<Geometry, String>builder()
-        .put(A, "A")
-        .put(B, "B")
-        .put(C, "C")
-        .put(D, "D")
-        .put(E, "E")
-        .put(F, "F")
-        .put(G, "G")
-        .build();
-
-    /**
-     * JUnit 4 parameterized iterates thru this list and calls the constructor with each.
-     * For each test, Call the constructor three times, for WKT and for GML encoding 1, and GML encoding 2
-     */
-    private static final URI USE_JTS_LIB_ENCODING = new URIImpl("uri:useLib") ;
-    private static final URI USE_ROUGH_ENCODING = new URIImpl("uri:useRough") ;
-
-    @Parameters
-    public static Collection<URI[]> constructorData() {
-        final URI[][] data = new URI[][] { { GeoConstants.XMLSCHEMA_OGC_WKT, USE_JTS_LIB_ENCODING }, { GeoConstants.XMLSCHEMA_OGC_GML, USE_JTS_LIB_ENCODING }, { GeoConstants.XMLSCHEMA_OGC_GML, USE_ROUGH_ENCODING } };
-        return Arrays.asList(data);
-    }
-
-    private final URI schemaToTest;
-    private final URI encodeMethod;
-
-    /**
-     * Constructor required by JUnit parameterized runner.  See {@link #constructorData()} for constructor values.
-     * @param schemaToTest the schema to test {@link URI}.
-     * @param encodeMethod the encode method {@link URI}.
-     */
-    public GeoIndexerSfTest(final URI schemaToTest, final URI encodeMethod) {
-        this.schemaToTest = schemaToTest;
-        this.encodeMethod = encodeMethod;
-    }
-
-    /**
-     * Run before each test method.
-     * @throws Exception
-     */
-    @Before
-    public void before() throws Exception {
-        conf = new AccumuloRdfConfiguration();
-        conf.setTablePrefix("triplestore_");
-        final String tableName = GeoMesaGeoIndexer.getTableName(conf);
-        conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, true);
-        conf.set(ConfigUtils.CLOUDBASE_USER, "USERNAME");
-        conf.set(ConfigUtils.CLOUDBASE_PASSWORD, "PASS");
-        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, "INSTANCE");
-        conf.set(ConfigUtils.CLOUDBASE_ZOOKEEPERS, "localhost");
-        conf.set(ConfigUtils.CLOUDBASE_AUTHS, "U");
-        conf.set(OptionalConfigUtils.USE_GEO, "true");
-        conf.set(OptionalConfigUtils.GEO_INDEXER_TYPE, GeoIndexerType.GEO_MESA.toString());
-
-        final TableOperations tops = ConfigUtils.getConnector(conf).tableOperations();
-        // get all of the table names with the prefix
-        final Set<String> toDel = Sets.newHashSet();
-        for (final String t : tops.list()) {
-            if (t.startsWith(tableName)) {
-                toDel.add(t);
-            }
-        }
-        for (final String t : toDel) {
-            tops.delete(t);
-        }
-
-        g = new GeoMesaGeoIndexer();
-        g.setConf(conf);
-        // Convert the statements as schema WKT or GML, then GML has two methods to encode.
-        g.storeStatement(createRyaStatement(A, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(B, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(C, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(D, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(F, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(E, schemaToTest, encodeMethod));
-        g.storeStatement(createRyaStatement(G, schemaToTest, encodeMethod));
-    }
-
-    private static RyaStatement createRyaStatement(final Geometry geo, final URI schema, final URI encodingMethod) {
-        return RdfToRyaConversions.convertStatement(genericStatement(geo,schema,encodingMethod));
-    }
-
-    private static Statement genericStatement(final Geometry geo, final URI schema, final URI encodingMethod) {
-        if (schema.equals(GeoConstants.XMLSCHEMA_OGC_WKT)) {
-            return genericStatementWkt(geo);
-        } else if (schema.equals(GeoConstants.XMLSCHEMA_OGC_GML)) {
-            return genericStatementGml(geo, encodingMethod);
-        }
-        throw new Error("schema unsupported: "+schema);
-    }
-
-    private static Statement genericStatementWkt(final Geometry geo) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("uri:" + NAMES.get(geo));
-        final URI predicate = GeoConstants.GEO_AS_WKT;
-        final Value object = vf.createLiteral(geo.toString(), GeoConstants.XMLSCHEMA_OGC_WKT);
-        return new StatementImpl(subject, predicate, object);
-    }
-
-    private static Statement genericStatementGml(final Geometry geo, final URI encodingMethod) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("uri:" + NAMES.get(geo));
-        final URI predicate = GeoConstants.GEO_AS_GML;
-
-        final String gml ;
-        if (encodingMethod == USE_JTS_LIB_ENCODING) {
-            gml = geoToGmlUseJtsLib(geo);
-        } else if (encodingMethod == USE_ROUGH_ENCODING) {
-            gml = geoToGmlRough(geo);
-        }
-        else {
-            throw new Error("invalid encoding method: "+encodingMethod);
-        //        System.out.println("===created GML====");
-        //        System.out.println(gml);
-        //        System.out.println("========== GML====");
-        }
-
-        final Value object = vf.createLiteral(gml, GeoConstants.XMLSCHEMA_OGC_GML);
-        return new StatementImpl(subject, predicate, object);
-    }
-
-    /**
-     * JTS library conversion from geometry to GML.
-     * @param geo base Geometry gets delegated
-     * @return String gml encoding of the geomoetry
-     */
-    private static String geoToGmlUseJtsLib(final Geometry geo) {
-        final int srid = geo.getSRID();
-        final GMLWriter gmlWriter = new GMLWriter();
-        gmlWriter.setNamespace(false);
-        gmlWriter.setPrefix(null);
-
-        if (srid != -1 || srid != 0) {
-            gmlWriter.setSrsName("EPSG:" + geo.getSRID());
-        }
-        final String gml = gmlWriter.write(geo);
-        // Hack to replace a gml 2.0 deprecated element in the Polygon.
-        // It should tolerate this as it does other depreciated elements like <gml:coordinates>.
-        return gml.replace("outerBoundaryIs", "exterior");
-    }
-
-    /**
-     * Rough conversion from geometry to GML using a template.
-     * @param geo base Geometry gets delegated
-     * @return String gml encoding of the gemoetry
-     */
-    private static String geoToGmlRough(final Geometry geo) {
-        final Geometries theType = org.geotools.geometry.jts.Geometries.get(geo);
-        switch (theType) {
-        case POINT:
-            return geoToGml((Point)geo);
-        case LINESTRING:
-            return geoToGml((LineString)geo);
-        case POLYGON:
-            return geoToGml((Polygon)geo);
-        case MULTIPOINT:
-        case MULTILINESTRING:
-        case MULTIPOLYGON:
-        default:
-            throw new Error("No code to convert to GML for this type: "+theType);
-        }
-    }
-
-    private static Point point(final double x, final double y) {
-        return gf.createPoint(new Coordinate(x, y));
-    }
-
-    private static String geoToGml(final Point point) {
-        //CRS:84 long X,lat Y
-        //ESPG:4326 lat Y,long X
-        return "<Point"//
-        + " srsName='CRS:84'"// TODO: point.getSRID()
-        + "><pos>"+point.getX()+" "+point.getY()+"</pos>  "// assumes  Y=lat  X=long
-        + " </Point>";
-    }
-
-    private static LineString line(final double x1, final double y1, final double x2, final double y2) {
-        return new LineString(new PackedCoordinateSequence.Double(new double[] { x1, y1, x2, y2 }, 2), gf);
-    }
-
-    /**
-     * convert a lineString geometry to GML
-     * @param line
-     * @return String that is XML that is a GMLLiteral of line
-     */
-    private static String geoToGml(final LineString line) {
-        final StringBuilder coordString = new StringBuilder() ;
-        for (final Coordinate coor : line.getCoordinates()) {
-            coordString.append(" ").append(coor.x).append(" ").append(coor.y); //ESPG:4326 lat/long
-        }
-        return " <gml:LineString srsName=\"http://www.opengis.net/def/crs/EPSG/0/4326\" xmlns:gml='http://www.opengis.net/gml'>\n"
-                + "<gml:posList srsDimension=\"2\">"//
-                + coordString //
-                + "</gml:posList></gml:LineString >";
-    }
-
-    private static Polygon poly(final double[] arr) {
-        final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(arr, 2));
-        final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-        return p1;
-    }
-
-    /**
-     * convert a Polygon geometry to GML
-     * @param geometry
-     * @return String that is XML that is a GMLLiteral of line
-     */
-    private static String geoToGml(final Polygon poly) {
-        final StringBuilder coordString = new StringBuilder() ;
-        for (final Coordinate coor : poly.getCoordinates()) {
-            coordString.append(" ").append(coor.x).append(" ").append(coor.y); //ESPG:4326 lat/long
-            //with commas:  coordString.append(" ").append(coor.x).append(",").append(coor.y);
-        }
-        return "<gml:Polygon srsName=\"EPSG:4326\"  xmlns:gml='http://www.opengis.net/gml'>\r\n"//
-                + "<gml:exterior><gml:LinearRing>\r\n"//
-                + "<gml:posList srsDimension='2'>\r\n"
-                +  coordString
-                + "</gml:posList>\r\n"//
-                + "</gml:LinearRing></gml:exterior>\r\n</gml:Polygon>\r\n";
-    }
-
-    private static double[] bbox(final double x1, final double y1, final double x2, final double y2) {
-        return new double[] { x1, y1, x1, y2, x2, y2, x2, y1, x1, y1 };
-    }
-
-    private void compare(final CloseableIteration<Statement, ?> actual, final Geometry... expected) throws Exception {
-        final Set<Statement> expectedSet = Sets.newHashSet();
-        for (final Geometry geo : expected) {
-            expectedSet.add(RyaToRdfConversions.convertStatement(createRyaStatement(geo, this.schemaToTest, encodeMethod)));
-        }
-
-        Assert.assertEquals(expectedSet, getSet(actual));
-    }
-
-    private static final Geometry[] EMPTY_RESULTS = {};
-
-    @Test
-    public void testParsePoly() throws Exception {
-        assertParseable(D);
-    }
-
-    @Test
-    public void testParseLine() throws Exception {
-        assertParseable(E);
-    }
-
-    @Test
-    public void testParsePoint() throws Exception {
-        assertParseable(F);
-    }
-
-    /**
-     * Convert Geometry to Wkt|GML (schemaToTest), parse to Geometry, and compare to original.
-     * @param originalGeom the original {@link Geometry}.
-     * @throws ParseException
-     */
-    public void assertParseable(final Geometry originalGeom) throws ParseException {
-        final Geometry parsedGeom = GeoParseUtils.getGeometry(genericStatement(originalGeom,schemaToTest, encodeMethod));
-        assertTrue("Parsed should equal original: "+originalGeom+" parsed: "+parsedGeom, originalGeom.equalsNorm(parsedGeom));
-        assertEquals( originalGeom, parsedGeom ); //also passes
-        assertTrue( originalGeom.equalsExact(parsedGeom) ); //also passes
-    }
-
-    @Test
-    public void testEquals() throws Exception {
-        // point
-        compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
-        compare(g.queryEquals(point(-1, -1), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-        // line
-        compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
-        compare(g.queryEquals(line(-1, -1, 0, 0), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-        // poly
-        compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
-        compare(g.queryEquals(poly(bbox(-2, -2, 1, 2)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
-
-    @Test
-    public void testDisjoint() throws Exception {
-        // point
-        compare(g.queryDisjoint(F, EMPTY_CONSTRAINTS), B, C, D, E, G);
-
-        // line
-        compare(g.queryDisjoint(E, EMPTY_CONSTRAINTS), B, C, F, G);
-
-        // poly
-        compare(g.queryDisjoint(A, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-        compare(g.queryDisjoint(B, EMPTY_CONSTRAINTS), C, D, F, E, G);
-    }
-
-    @Test
-    @Ignore
-    public void testIntersectsPoint() throws Exception {
-        // This seems like a bug
-        //   scala.MatchError: POINT (2 4) (of class com.vividsolutions.jts.geom.Point)
-        //   at org.locationtech.geomesa.filter.FilterHelper$.updateToIDLSafeFilter(FilterHelper.scala:53)
-        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), A, F);
-        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
-
-    @Ignore
-    @Test
-    public void testIntersectsLine() throws Exception {
-        // This seems like a bug
-        // fails with:
-        //     scala.MatchError: LINESTRING (2 0, 3 3) (of class com.vividsolutions.jts.geom.LineString)
-        //     at org.locationtech.geomesa.filter.FilterHelper$.updateToIDLSafeFilter(FilterHelper.scala:53)
-        //compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), A, E, D);
-        //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, G);
-    }
-
-    @Test
-    public void testTouchesPoint() throws Exception {
-        compare(g.queryTouches(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-        compare(g.queryTouches(G, EMPTY_CONSTRAINTS), A, C);
-    }
-
-    @Test
-    public void testTouchesLine() throws Exception {
-        compare(g.queryTouches(E, EMPTY_CONSTRAINTS), D);
-    }
-
-    @Test
-    public void testTouchesPoly() throws Exception {
-        compare(g.queryTouches(A, EMPTY_CONSTRAINTS), C,G);
-    }
-
-    @Test
-    public void testCrossesPoint() throws Exception {
-        compare(g.queryCrosses(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-        compare(g.queryCrosses(G, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-        // bug? java.lang.IllegalStateException:  getX called on empty Point
-        //    compare(g.queryCrosses(point(2, 0), EMPTY_CONSTRAINTS), E);
-    }
-
-    @Ignore
-    @Test
-    public void testCrossesLine() throws Exception {
-        // fails with:
-        //     java.lang.IllegalStateException: getX called on empty Point
-        //      at com.vividsolutions.jts.geom.Point.getX(Point.java:124)
-        //      at org.locationtech.geomesa.utils.geohash.GeohashUtils$.considerCandidate$1(GeohashUtils.scala:1023)
-
-        // compare(g.queryCrosses(E, EMPTY_CONSTRAINTS), A);
-    }
-
-    @Test
-    public void testCrossesPoly() throws Exception {
-        compare(g.queryCrosses(A, EMPTY_CONSTRAINTS), E);
-        compare(g.queryCrosses(poly(bbox(-0.9, -2.9, -0.1, -1.1)), EMPTY_CONSTRAINTS), E);
-    }
-
-    @Test
-    public void testWithin() throws Exception {
-        // point
-        // geomesa bug? scala.MatchError: POINT (2 4) (of class com.vividsolutions.jts.geom.Point)
-        //    compare(g.queryWithin(F, EMPTY_CONSTRAINTS), F);
-
-        // line
-        // geomesa bug? scala.MatchError: LINESTRING (2 0, 3 2) (of class com.vividsolutions.jts.geom.LineString)
-        //    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);
-    }
-
-    @Ignore
-    @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);
-    }
-
-    @Ignore
-    @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);
-    }
-
-    @Ignore
-    @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);
-    }
-
-}