You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by pu...@apache.org on 2016/12/15 18:40:00 UTC

[06/10] incubator-rya git commit: RYA-127 Mongo support in merge tool. (#14)

RYA-127 Mongo support in merge tool. (#14)

Added full mongo support to the merge tool.
This includes unit tests.


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

Branch: refs/heads/master
Commit: 2c37b24546e3b6b642bf58f95c5a5974cd8c3d92
Parents: 24c7183
Author: Andrew Smith <sm...@gmail.com>
Authored: Thu Sep 8 13:08:25 2016 -0400
Committer: isper3at <sm...@gmail.com>
Committed: Wed Nov 2 17:51:18 2016 -0400

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 extras/rya.export/export.mongo/pom.xml          | 103 ++++++++++++++++
 .../export/mongo/MongoRyaStatementStore.java    | 118 +++++++++++++++++++
 .../mongo/MongoRyaStatementStoreDecorator.java  |  46 ++++++++
 .../parent/MongoParentMetadataRepository.java   |  69 +++++++++++
 .../parent/ParentMetadataRepositoryAdapter.java |  68 +++++++++++
 .../mongo/time/TimeMongoRyaStatementStore.java  | 101 ++++++++++++++++
 .../ParentMetadataRepositoryAdapterTest.java    |  95 +++++++++++++++
 extras/rya.export/pom.xml                       |   1 +
 9 files changed, 602 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index cb7f0ff..690fcd7 100644
--- a/.gitignore
+++ b/.gitignore
@@ -61,3 +61,4 @@ MANIFEST.MF
 delme*
 
 **/.vagrant/
+**/src/gen/**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/pom.xml b/extras/rya.export/export.mongo/pom.xml
new file mode 100644
index 0000000..e350a92
--- /dev/null
+++ b/extras/rya.export/export.mongo/pom.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+     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.
+-->
+<project 
+    xmlns="http://maven.apache.org/POM/4.0.0" 
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.export.parent</artifactId>
+        <version>3.2.10-SNAPSHOT</version>
+    </parent>
+    
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rya.export.mongo</artifactId>
+    
+    <name>Apache Rya Export Mongo</name>
+    <description>Contains the mongo implementation of the export tool.</description>
+    
+    <dependencies>
+        <!-- Rya Runtime Dependencies. -->
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.export.api</artifactId>
+            <version>3.2.10-SNAPSHOT</version>
+        </dependency>
+        
+        <!-- Log4j 2 bridge, api, and core. -->        
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-1.2-api</artifactId>
+            <version>2.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+            <version>2.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <version>2.5</version>
+        </dependency> 
+        
+        <!-- Testing dependencies. -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    
+    <build>
+        <plugins>
+            <!-- Use the pre-build 'jar-with-dependencies' assembly to package the dependent class files into the final jar. 
+                                  This creates a jar file that can be deployed to Fluo without having to include any dependent jars. -->
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <archive>
+                        <manifest>
+                            <addClasspath>true</addClasspath>
+                            <classpathLayoutType>custom</classpathLayoutType>
+                            <customClasspathLayout>WEB-INF/lib/$${artifact.groupIdPath}/$${artifact.artifactId}-$${artifact.version}$${dashClassifier?}.$${artifact.extension}</customClasspathLayout>
+                        
+                            <mainClass>org.apache.rya.indexing.pcj.fluo.PcjAdminClient</mainClass>
+                        </manifest>
+                    </archive>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
+

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStore.java b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStore.java
new file mode 100644
index 0000000..ed24080
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStore.java
@@ -0,0 +1,118 @@
+/*
+ * 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.export.mongo;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static mvm.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.rya.export.api.store.AddStatementException;
+import org.apache.rya.export.api.store.ContainsStatementException;
+import org.apache.rya.export.api.store.RemoveStatementException;
+import org.apache.rya.export.api.store.RyaStatementStore;
+import org.apache.rya.export.api.store.UpdateStatementException;
+
+import com.mongodb.BasicDBObject;
+import com.mongodb.Cursor;
+import com.mongodb.DB;
+import com.mongodb.DBObject;
+import com.mongodb.MongoClient;
+
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.mongodb.MongoDBRyaDAO;
+import mvm.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
+
+/**
+ * Mongo implementation of {@link RyaStatementStore}.  Allows for exporting and
+ * importing rya statements from MongoDB.
+ */
+public class MongoRyaStatementStore implements RyaStatementStore{
+    public static final String TRIPLES_COLLECTION = "rya__triples";
+    private final SimpleMongoDBStorageStrategy adapter;
+    private final DB db;
+
+    private final String ryaInstanceName;
+    private final MongoClient client;
+    private final MongoDBRyaDAO dao;
+
+    /**
+     * Creates a new {@link MongoRyaStatementStore}.
+     * @param client - The client to connect to Mongo. (not null)
+     * @param ryaInstance - The rya instance to connect to. (not null)
+     * @param dao - The {@link MongoDBRyaDAO} to use to access statements. (not null)
+     */
+    public MongoRyaStatementStore(final MongoClient client, final String ryaInstance, final MongoDBRyaDAO dao) {
+        this.client = checkNotNull(client);
+        ryaInstanceName = checkNotNull(ryaInstance);
+        this.dao = checkNotNull(dao);
+        db = this.client.getDB(ryaInstanceName);
+        adapter = new SimpleMongoDBStorageStrategy();
+    }
+
+    @Override
+    public Iterator<RyaStatement> fetchStatements() {
+        final Cursor cur = db.getCollection(TRIPLES_COLLECTION).find().sort(new BasicDBObject(TIMESTAMP, 1));
+        final List<RyaStatement> statements = new ArrayList<>();
+        while(cur.hasNext()) {
+            final RyaStatement statement = adapter.deserializeDBObject(cur.next());
+            statements.add(statement);
+        }
+        return statements.iterator();
+    }
+
+    @Override
+    public void addStatement(final RyaStatement statement) throws AddStatementException {
+        try {
+            dao.add(statement);
+        } catch (final RyaDAOException e) {
+            throw new AddStatementException("Unable to add statement: '" + statement.toString() + "'", e);
+        }
+    }
+
+    @Override
+    public void removeStatement(final RyaStatement statement) throws RemoveStatementException {
+        try {
+            //mongo dao does not need a config to remove.
+            dao.delete(statement, null);
+        } catch (final RyaDAOException e) {
+            throw new RemoveStatementException("Unable to remove statement: '" + statement.toString() + "'", e);
+        }
+    }
+
+    @Override
+    public boolean containsStatement(final RyaStatement statement) throws ContainsStatementException {
+        final DBObject dbo = adapter.serialize(statement);
+        return db.getCollection(TRIPLES_COLLECTION).find(dbo).count() > 0;
+    }
+
+    protected MongoClient getClient() {
+        return client;
+    }
+
+    @Override
+    public void updateStatement(final RyaStatement original, final RyaStatement update) throws UpdateStatementException {
+        //Since mongo does not support visibility, this does nothing for mongo.
+        //Do not want a throw a not-implemented exception since that could potentially
+        //break stuff.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStoreDecorator.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStoreDecorator.java b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStoreDecorator.java
new file mode 100644
index 0000000..6f02b44
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/MongoRyaStatementStoreDecorator.java
@@ -0,0 +1,46 @@
+/*
+ * 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.export.mongo;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.rya.export.api.store.RyaStatementStore;
+import org.apache.rya.export.api.store.RyaStatementStoreDecorator;
+
+import com.mongodb.MongoClient;
+
+/**
+ * Ensures the decorator that the decorated store is mongodb backed.
+ */
+public abstract class MongoRyaStatementStoreDecorator extends RyaStatementStoreDecorator {
+    final MongoRyaStatementStore store;
+
+    /**
+     * Creates a new {@link MongoRyaStatementStoreDecorator} around the provided {@link RyaStatementStore}.
+     * @param store - The {@link RyaStatementStore} to decorate.
+     */
+    public MongoRyaStatementStoreDecorator(final MongoRyaStatementStore store) {
+        super(store);
+        this.store = checkNotNull(store);
+    }
+
+    protected MongoClient getClient() {
+        return store.getClient();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/MongoParentMetadataRepository.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/MongoParentMetadataRepository.java b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/MongoParentMetadataRepository.java
new file mode 100644
index 0000000..84471d3
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/MongoParentMetadataRepository.java
@@ -0,0 +1,69 @@
+/*
+ * 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.export.mongo.parent;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.rya.export.api.parent.MergeParentMetadata;
+import org.apache.rya.export.api.parent.ParentMetadataDoesNotExistException;
+import org.apache.rya.export.api.parent.ParentMetadataExistsException;
+import org.apache.rya.export.api.parent.ParentMetadataRepository;
+
+import com.mongodb.DBCollection;
+import com.mongodb.DBObject;
+import com.mongodb.MongoClient;
+
+/**
+ * Repository for storing the {@link MergeParentMetadata}.
+ */
+public class MongoParentMetadataRepository implements ParentMetadataRepository {
+    private static final String COLLECTION_NAME = "parent_metadata";
+    private final ParentMetadataRepositoryAdapter adapter;
+    private final DBCollection collection;
+
+    /**
+     * @param client
+     * @param dbName
+     */
+    public MongoParentMetadataRepository(final MongoClient client, final String dbName) {
+        checkNotNull(client);
+        checkNotNull(dbName);
+        collection = client.getDB(dbName).getCollection(COLLECTION_NAME);
+        adapter = new ParentMetadataRepositoryAdapter();
+    }
+
+    @Override
+    public MergeParentMetadata get() throws ParentMetadataDoesNotExistException {
+        final DBObject mongoMetadata = collection.findOne();
+        if(mongoMetadata == null) {
+            throw new ParentMetadataDoesNotExistException("The parent metadata has not been set.");
+        }
+        return adapter.deserialize(mongoMetadata);
+    }
+
+    @Override
+    public void set(final MergeParentMetadata metadata) throws ParentMetadataExistsException {
+        if(collection.getCount() > 0) {
+            throw new ParentMetadataExistsException("The parent metadata has already been set.");
+        }
+        final DBObject dbo = adapter.serialize(metadata);
+        collection.insert(dbo);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapter.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapter.java b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapter.java
new file mode 100644
index 0000000..c6f2401
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapter.java
@@ -0,0 +1,68 @@
+/*
+ * 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.export.mongo.parent;
+
+import java.util.Date;
+
+import org.apache.rya.export.api.parent.MergeParentMetadata;
+
+import com.mongodb.BasicDBObjectBuilder;
+import com.mongodb.DBObject;
+
+/**
+ * Adapter for converting {@link MergeParentMetadata} to and from mongo
+ * {@link DBObject}s.
+ */
+public class ParentMetadataRepositoryAdapter {
+    public static final String RYANAME_KEY = "ryaInstanceName";
+    public static final String TIMESTAMP_KEY = "timestamp";
+    public static final String FILTER_TIMESTAMP_KEY = "filterTimestamp";
+    public static final String PARENT_TIME_OFFSET_KEY = "parentTimeOffset";
+    /**
+     * Serializes the {@link MergeParentMetadata} into a mongoDB object.
+     * @param metadata - The {@link MergeParentMetadata} to serialize.
+     * @return The MongoDB object
+     */
+    public DBObject serialize(final MergeParentMetadata metadata) {
+        final BasicDBObjectBuilder builder = BasicDBObjectBuilder.start()
+            .add(RYANAME_KEY, metadata.getRyaInstanceName())
+            .add(TIMESTAMP_KEY, metadata.getTimestamp())
+            .add(FILTER_TIMESTAMP_KEY, metadata.getFilterTimestamp())
+            .add(PARENT_TIME_OFFSET_KEY, metadata.getParentTimeOffset());
+        return builder.get();
+    }
+
+    /**
+     * Deserialize the mongoBD object into {@link MergeParentMetadata}.
+     * @param dbo - The mongo {@link DBObject} to deserialize.
+     * @return The {@link MergeParentMetadata}
+     */
+    public MergeParentMetadata deserialize(final DBObject dbo) {
+        final Date timestamp = (Date) dbo.get(TIMESTAMP_KEY);
+        final String ryaInstance = (String) dbo.get(RYANAME_KEY);
+        final Date filterTimestamp = (Date) dbo.get(FILTER_TIMESTAMP_KEY);
+        final Long offset = (Long) dbo.get(PARENT_TIME_OFFSET_KEY);
+        return new MergeParentMetadata.Builder()
+            .setRyaInstanceName(ryaInstance)
+            .setTimestamp(timestamp)
+            .setFilterTimestmap(filterTimestamp)
+            .setParentTimeOffset(offset)
+            .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/time/TimeMongoRyaStatementStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/time/TimeMongoRyaStatementStore.java b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/time/TimeMongoRyaStatementStore.java
new file mode 100644
index 0000000..4c948ed
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/main/java/org/apache/rya/export/mongo/time/TimeMongoRyaStatementStore.java
@@ -0,0 +1,101 @@
+/*
+ * 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.export.mongo.time;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static mvm.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.rya.export.mongo.MongoRyaStatementStore;
+import org.apache.rya.export.mongo.MongoRyaStatementStoreDecorator;
+
+import com.mongodb.BasicDBObject;
+import com.mongodb.Cursor;
+import com.mongodb.DB;
+
+import mvm.rya.api.domain.RyaStatement;
+import mvm.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
+
+/**
+ * A {@link MongoRyaStatementStore} that, when fetching statements, only
+ * fetches statements after a certain time.
+ */
+public class TimeMongoRyaStatementStore extends MongoRyaStatementStoreDecorator {
+    private final Date time;
+    private final DB db;
+
+    private final SimpleMongoDBStorageStrategy adapter;
+
+    /**
+     * Creates a new {@link TimeMongoRyaStatementStore}.
+     * @param store - The {@link MongoRyaStatementStore} to decorate.
+     * @param time - The time used when fetching statements.
+     * @param ryaInstanceName - The rya instance used.
+     */
+    public TimeMongoRyaStatementStore(final MongoRyaStatementStore store, final Date time, final String ryaInstanceName) {
+        super(store);
+        this.time = checkNotNull(time);
+        db = getClient().getDB(ryaInstanceName);
+        adapter = new SimpleMongoDBStorageStrategy();
+    }
+
+    /**
+     * @return
+     * @see org.apache.rya.export.mongo.MongoRyaStatementStore#fetchStatements()
+     */
+    @Override
+    public Iterator<RyaStatement> fetchStatements() {
+        //RyaStatement timestamps are stored as longs, not dates.
+        final BasicDBObject dbo = new BasicDBObject(TIMESTAMP, new BasicDBObject("$gte", time.getTime()));
+        final Cursor cur = db.getCollection(MongoRyaStatementStore.TRIPLES_COLLECTION).find(dbo).sort(new BasicDBObject(TIMESTAMP, 1));
+        final List<RyaStatement> statements = new ArrayList<>();
+        while(cur.hasNext()) {
+            final RyaStatement statement = adapter.deserializeDBObject(cur.next());
+            statements.add(statement);
+        }
+        return statements.iterator();
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if(obj instanceof TimeMongoRyaStatementStore) {
+            final TimeMongoRyaStatementStore other = (TimeMongoRyaStatementStore) obj;
+            final EqualsBuilder builder = new EqualsBuilder()
+                .appendSuper(super.equals(obj))
+                .append(time, other.time);
+            return builder.isEquals();
+        }
+        return false;
+    }
+
+
+    @Override
+    public int hashCode() {
+        final HashCodeBuilder builder = new HashCodeBuilder()
+            .appendSuper(super.hashCode())
+            .append(time);
+        return builder.toHashCode();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/export.mongo/src/test/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapterTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.mongo/src/test/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapterTest.java b/extras/rya.export/export.mongo/src/test/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapterTest.java
new file mode 100644
index 0000000..d5d4757
--- /dev/null
+++ b/extras/rya.export/export.mongo/src/test/java/org/apache/rya/export/mongo/parent/ParentMetadataRepositoryAdapterTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.export.mongo.parent;
+
+import static org.apache.rya.export.mongo.parent.ParentMetadataRepositoryAdapter.FILTER_TIMESTAMP_KEY;
+import static org.apache.rya.export.mongo.parent.ParentMetadataRepositoryAdapter.PARENT_TIME_OFFSET_KEY;
+import static org.apache.rya.export.mongo.parent.ParentMetadataRepositoryAdapter.RYANAME_KEY;
+import static org.apache.rya.export.mongo.parent.ParentMetadataRepositoryAdapter.TIMESTAMP_KEY;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Date;
+
+import org.apache.rya.export.api.parent.MergeParentMetadata;
+import org.junit.Test;
+
+import com.mongodb.BasicDBObjectBuilder;
+import com.mongodb.DBObject;
+
+public class ParentMetadataRepositoryAdapterTest {
+    private final static String TEST_INSTANCE = "test_instance";
+    private final static Date TEST_TIMESTAMP = new Date(8675309L);
+    private final static Date TEST_FILTER_TIMESTAMP = new Date(1234567L);
+    private final static long TEST_TIME_OFFSET = 123L;
+    private final ParentMetadataRepositoryAdapter adapter = new ParentMetadataRepositoryAdapter();
+
+    @Test
+    public void deserializeTest() {
+        final DBObject dbo = BasicDBObjectBuilder.start()
+            .add(RYANAME_KEY, TEST_INSTANCE)
+            .add(TIMESTAMP_KEY, TEST_TIMESTAMP)
+            .add(FILTER_TIMESTAMP_KEY, TEST_FILTER_TIMESTAMP)
+            .add(PARENT_TIME_OFFSET_KEY, TEST_TIME_OFFSET)
+            .get();
+
+        final MergeParentMetadata expected = new MergeParentMetadata.Builder()
+            .setRyaInstanceName(TEST_INSTANCE)
+            .setTimestamp(TEST_TIMESTAMP)
+            .setFilterTimestmap(TEST_FILTER_TIMESTAMP)
+            .setParentTimeOffset(TEST_TIME_OFFSET)
+            .build();
+        final MergeParentMetadata actual = adapter.deserialize(dbo);
+        assertEquals(expected, actual);
+    }
+
+    @Test(expected=NullPointerException.class)
+    public void deserializeTest_missingTime() {
+        final DBObject dbo = BasicDBObjectBuilder.start()
+            .add(RYANAME_KEY, TEST_INSTANCE)
+            .get();
+        adapter.deserialize(dbo);
+    }
+
+    @Test(expected=NullPointerException.class)
+    public void deserializeTest_missingName() {
+        final DBObject dbo = BasicDBObjectBuilder.start()
+            .add(TIMESTAMP_KEY, TEST_TIMESTAMP)
+            .get();
+        adapter.deserialize(dbo);
+    }
+
+    @Test
+    public void serializeTest() {
+        final MergeParentMetadata merge = new MergeParentMetadata.Builder()
+            .setRyaInstanceName(TEST_INSTANCE)
+            .setTimestamp(TEST_TIMESTAMP)
+            .setFilterTimestmap(TEST_FILTER_TIMESTAMP)
+            .setParentTimeOffset(TEST_TIME_OFFSET)
+            .build();
+
+        final DBObject expected = BasicDBObjectBuilder.start()
+            .add(RYANAME_KEY, TEST_INSTANCE)
+            .add(TIMESTAMP_KEY, TEST_TIMESTAMP)
+            .add(FILTER_TIMESTAMP_KEY, TEST_FILTER_TIMESTAMP)
+            .add(PARENT_TIME_OFFSET_KEY, TEST_TIME_OFFSET)
+            .get();
+        final DBObject actual = adapter.serialize(merge);
+        assertEquals(expected, actual);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2c37b245/extras/rya.export/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.export/pom.xml b/extras/rya.export/pom.xml
index 92c715c..b2fd1d1 100644
--- a/extras/rya.export/pom.xml
+++ b/extras/rya.export/pom.xml
@@ -38,6 +38,7 @@ under the License.
    
     <modules>
         <module>export.api</module>
+        <module>export.mongo</module>
     </modules>
     
     <properties>