You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by kchilton2 <gi...@git.apache.org> on 2017/12/27 19:48:34 UTC

[GitHub] incubator-rya pull request #256: RYA-414

GitHub user kchilton2 opened a pull request:

    https://github.com/apache/incubator-rya/pull/256

    RYA-414

    ## Description
    A Mongo Sail object has been updated to only use a single MongoClient and only single instances of each indexer. By doing this, we can have multiple sail objects in memory that are able to talk to different MongoDB based Rya instances. For example, we could auth using the DB administrator in the Mongo Shell as well as create a sail object as a user that is able to query a specific instance within the same JVM at the same time.
    
    ### Tests
    Updated tests that were effected by the refactor.
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kchilton2/incubator-rya RYA-415

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-rya/pull/256.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #256
    
----
commit 767349dac9822cd13e92f9b117d1b5d2dad13e3d
Author: kchilton2 <ke...@...>
Date:   2017-12-23T01:52:27Z

    RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.

commit 17cebae3328916bd80fbe5447da5ccb660539556
Author: Andrew Smith <sm...@...>
Date:   2017-12-26T19:30:32Z

    RYA-414 Removed mongo connection factory
    
    addressed the indexers that used the factory
    addressed the geo project

commit 157c06491cd814a1d6e445ebfe77fc63226e5739
Author: kchilton2 <ke...@...>
Date:   2017-12-26T23:42:33Z

    RYA-414 Fixing broken tests, cleaning up documentate, cleaning up whitespace.

----


---

[GitHub] incubator-rya issue #256: RYA-414

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/incubator-rya/pull/256
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/578/



---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159479656
  
    --- Diff: extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java ---
    @@ -36,7 +36,7 @@
     import org.openrdf.model.ValueFactory;
     import org.openrdf.model.impl.ValueFactoryImpl;
     
    -public class GeoTemporalProviderTest extends GeoTemporalTestBase {
    +public class GeoTemporalProviderTest extends GeoTemporalTestUtils {
    --- End diff --
    
    Do we still need to extend GeoTemporalTestUtils if all the functions are now public static?


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159477674
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java ---
    @@ -120,7 +120,7 @@ public B setMongoDBName(String name) {
          * Sets MongoDB Collection prefix. This parameter must be set to connect to
          * an instance of MongoDB and will default to "rya_" is no value is
          * specified.
    -     * 
    +     *
          * @param name - name of Collection to connect to
    --- End diff --
    
    should be @param prefix


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159509885
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java ---
    @@ -79,19 +79,27 @@ private int findRandomOpenPortOnAllLocalInterfaces() throws IOException {
     
         /**
          * Creates a new Mongo connection.
    -     * 
    +     *
          * @throws MongoException
          * @throws UnknownHostException
          */
         public MongoClient newMongoClient() throws UnknownHostException, MongoException {
             return new MongoClient(new ServerAddress(mongodProcess.getConfig().net().getServerAddress(), mongodProcess.getConfig().net().getPort()));
         }
     
    +    /**
    +     * Gives access to the process configuration.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159309851
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java ---
    @@ -19,20 +19,43 @@
     package org.apache.rya.mongodb;
     
     import java.io.IOException;
    +import java.net.UnknownHostException;
     
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import com.mongodb.MongoClient;
    +import com.mongodb.MongoException;
    +
    +import de.flapdoodle.embed.mongo.config.IMongodConfig;
     
     /**
      * To be used for tests. Creates a singleton {@link MongoClient} to be used
      * throughout all of the MongoDB related tests. Without the singleton, the
      * embedded mongo factory ends up orphaning processes, consuming resources.
      */
     public class EmbeddedMongoSingleton {
    -    public static MongoClient getInstance() {
    -        return InstanceHolder.SINGLETON.instance;
    +
    +    public static MongoClient getNewMongoClient() throws UnknownHostException, MongoException {
    +    	final MongoClient client = InstanceHolder.SINGLETON.factory.newMongoClient();
    +
    +        Runtime.getRuntime().addShutdownHook(new Thread() {
    +            @Override
    +            public void run() {
    +                try {
    +                    client.close();
    +                } catch (final Throwable t) {
    +                    // logging frameworks will likely be shut down
    +                    t.printStackTrace(System.err);
    +                }
    +            }
    +        });
    +
    +        return client;
    +    }
    +
    +    public static IMongodConfig getMongodConfig() {
    --- End diff --
    
    javadocs


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159513534
  
    --- Diff: sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java ---
    @@ -2,12 +2,6 @@
     
     import static com.google.common.base.Preconditions.checkNotNull;
     
    -import org.openrdf.model.ValueFactory;
    -import org.openrdf.model.impl.ValueFactoryImpl;
    -import org.openrdf.sail.SailConnection;
    -import org.openrdf.sail.SailException;
    -import org.openrdf.sail.helpers.SailBase;
    -
     /*
      * Licensed to the Apache Software Foundation (ASF) under one
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159511388
  
    --- Diff: extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java ---
    @@ -33,15 +32,15 @@
     	/**
     	 * initialize after setting configuration.
     	 */
    -    public void init();  
    +    @Override
    +	public void init();  
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512111
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java ---
    @@ -92,184 +93,162 @@ public MongoDBRdfConfiguration clone() {
             return new MongoDBRdfConfiguration(this);
         }
     
    -    public Authorizations getAuthorizations() {
    -        final String[] auths = getAuths();
    -        if (auths == null || auths.length == 0) {
    -            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    -        }
    -        return new Authorizations(auths);
    -    }
    -
         /**
    -     * @return {@code true} if each statement added to the batch writer should
    -     * be flushed and written right away to the datastore. {@code false} if the
    -     * statements should be queued and written to the datastore when the queue
    -     * is full or after enough time has passed without a write.<p>
    -     * Defaults to {@code true} if nothing is specified.
    +     * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
          */
    -    public boolean flushEachUpdate(){
    -        return getBoolean(CONF_FLUSH_EACH_UPDATE, true);
    +    public void setUseMock(final boolean useMock) {
    +        this.setBoolean(USE_MOCK_MONGO, useMock);
         }
     
         /**
    -     * Sets the {@link #CONF_FLUSH_EACH_UPDATE} property of the configuration.
    -     * @param flush {@code true} if each statement added to the batch writer
    -     * should be flushed and written right away to the datastore. {@code false}
    -     * if the statements should be queued and written to the datastore when the
    -     * queue is full or after enough time has passed without a write.
    +     * Indicates whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @return {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159510572
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java ---
    @@ -19,20 +19,43 @@
     package org.apache.rya.mongodb;
     
     import java.io.IOException;
    +import java.net.UnknownHostException;
     
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import com.mongodb.MongoClient;
    +import com.mongodb.MongoException;
    +
    +import de.flapdoodle.embed.mongo.config.IMongodConfig;
     
     /**
      * To be used for tests. Creates a singleton {@link MongoClient} to be used
      * throughout all of the MongoDB related tests. Without the singleton, the
      * embedded mongo factory ends up orphaning processes, consuming resources.
      */
     public class EmbeddedMongoSingleton {
    -    public static MongoClient getInstance() {
    -        return InstanceHolder.SINGLETON.instance;
    +
    +    public static MongoClient getNewMongoClient() throws UnknownHostException, MongoException {
    +    	final MongoClient client = InstanceHolder.SINGLETON.factory.newMongoClient();
    +
    +        Runtime.getRuntime().addShutdownHook(new Thread() {
    +            @Override
    +            public void run() {
    +                try {
    +                    client.close();
    +                } catch (final Throwable t) {
    +                    // logging frameworks will likely be shut down
    +                    t.printStackTrace(System.err);
    +                }
    +            }
    +        });
    +
    +        return client;
    +    }
    +
    +    public static IMongodConfig getMongodConfig() {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512809
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java ---
    @@ -0,0 +1,102 @@
    +/**
    + * 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.mongodb;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
    + * {@link Configuration} objects.
    + * </p>
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512956
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java ---
    @@ -39,44 +35,16 @@
      */
     @DefaultAnnotation(NonNull.class)
     public class MongoEntityIndexer extends BaseEntityIndexer {
    -    private MongoClient client;
    -
    -    @Override
    -    public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoEntityStorage(client, ryaInstanceName);
    -    }
    -
    -    @Override
    -    public TypeStorage getTypeStorage(final Configuration conf) {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoTypeStorage(client, ryaInstanceName);
    -    }
    -
     
         @Override
    -    public void init() {
    -        //nothing to init.
    +    public EntityStorage getEntityStorage() throws EntityStorageException {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    +        return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstanceName());
         }
     
         @Override
    -    public void setClient(final MongoClient client) {
    -        this.client = client;
    +    public TypeStorage getTypeStorage() {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159312925
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java ---
    @@ -64,12 +60,13 @@
                     throw new RuntimeException(e);
                 }
                 return (RyaQueryEngine<C>) new AccumuloRyaQueryEngine(conn, aConf);
    -        } else if(conf instanceof MongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
    -            MongoClient client = MongoConnectorFactory.getMongoClient(conf);
    -            return (RyaQueryEngine<C>) new MongoDBQueryEngine((MongoDBRdfConfiguration) conf, client);
    +        } else if(conf instanceof StatefulMongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
    --- End diff --
    
    There's a constant for "sc.useMongo" inside ConfigUtils.  Maybe use that here or move that constant to MongoDBRdfConfiguration and use it.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159484331
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java ---
    @@ -39,44 +35,16 @@
      */
     @DefaultAnnotation(NonNull.class)
     public class MongoEntityIndexer extends BaseEntityIndexer {
    -    private MongoClient client;
    -
    -    @Override
    -    public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoEntityStorage(client, ryaInstanceName);
    -    }
    -
    -    @Override
    -    public TypeStorage getTypeStorage(final Configuration conf) {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoTypeStorage(client, ryaInstanceName);
    -    }
    -
     
         @Override
    -    public void init() {
    -        //nothing to init.
    +    public EntityStorage getEntityStorage() throws EntityStorageException {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    +        return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstanceName());
         }
     
         @Override
    -    public void setClient(final MongoClient client) {
    -        this.client = client;
    +    public TypeStorage getTypeStorage() {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    --- End diff --
    
    remove tab character


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159478708
  
    --- Diff: sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java ---
    @@ -2,12 +2,6 @@
     
     import static com.google.common.base.Preconditions.checkNotNull;
     
    -import org.openrdf.model.ValueFactory;
    -import org.openrdf.model.impl.ValueFactoryImpl;
    -import org.openrdf.sail.SailConnection;
    -import org.openrdf.sail.SailException;
    -import org.openrdf.sail.helpers.SailBase;
    -
     /*
      * Licensed to the Apache Software Foundation (ASF) under one
    --- End diff --
    
    nit.  move the header above the package line as was done in other files.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159511918
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java ---
    @@ -79,19 +79,27 @@ private int findRandomOpenPortOnAllLocalInterfaces() throws IOException {
     
         /**
          * Creates a new Mongo connection.
    -     * 
    +     *
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159482145
  
    --- Diff: extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java ---
    @@ -1,64 +0,0 @@
    -/**
    --- End diff --
    
    So it looks like you consolidated MongoITBase and MongoTestBase, which is great.  However, since MongoTestBase via EmbeddedMongoSingleton spawns new Mongo processes, it is by definition an Integration Test and should probably be renamed to MongoITBase for clarity.  Also, tests that use it (EntityQueryNodeTest, EventQueryNodeTest, MongoEventStorageTest, etc) should probably be renamed with an IT suffix so they can be exercised and cleaned up by the maven-failsafe-plugin.
      


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159308100
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java ---
    @@ -79,19 +79,27 @@ private int findRandomOpenPortOnAllLocalInterfaces() throws IOException {
     
         /**
          * Creates a new Mongo connection.
    -     * 
    +     *
          * @throws MongoException
          * @throws UnknownHostException
          */
         public MongoClient newMongoClient() throws UnknownHostException, MongoException {
             return new MongoClient(new ServerAddress(mongodProcess.getConfig().net().getServerAddress(), mongodProcess.getConfig().net().getPort()));
         }
     
    +    /**
    +     * Gives access to the process configuration.
    --- End diff --
    
    add @return


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159477077
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java ---
    @@ -92,184 +93,162 @@ public MongoDBRdfConfiguration clone() {
             return new MongoDBRdfConfiguration(this);
         }
     
    -    public Authorizations getAuthorizations() {
    -        final String[] auths = getAuths();
    -        if (auths == null || auths.length == 0) {
    -            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    -        }
    -        return new Authorizations(auths);
    -    }
    -
         /**
    -     * @return {@code true} if each statement added to the batch writer should
    -     * be flushed and written right away to the datastore. {@code false} if the
    -     * statements should be queued and written to the datastore when the queue
    -     * is full or after enough time has passed without a write.<p>
    -     * Defaults to {@code true} if nothing is specified.
    +     * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
    --- End diff --
    
    javadoc doesn't like {@true}


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512948
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java ---
    @@ -39,44 +35,16 @@
      */
     @DefaultAnnotation(NonNull.class)
     public class MongoEntityIndexer extends BaseEntityIndexer {
    -    private MongoClient client;
    -
    -    @Override
    -    public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoEntityStorage(client, ryaInstanceName);
    -    }
    -
    -    @Override
    -    public TypeStorage getTypeStorage(final Configuration conf) {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoTypeStorage(client, ryaInstanceName);
    -    }
    -
     
         @Override
    -    public void init() {
    -        //nothing to init.
    +    public EntityStorage getEntityStorage() throws EntityStorageException {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159510270
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java ---
    @@ -37,147 +37,145 @@
     
     public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
     
    -	public class NamespaceImplementation implements Namespace {
    +    public class NamespaceImplementation implements Namespace {
     
    -		private final String namespace;
    -		private final String prefix;
    +        private final String namespace;
    +        private final String prefix;
     
    -		public NamespaceImplementation(final String namespace, final String prefix) {
    -			this.namespace = namespace;
    -			this.prefix = prefix;
    -		}
    +        public NamespaceImplementation(final String namespace, final String prefix) {
    +            this.namespace = namespace;
    +            this.prefix = prefix;
    +        }
     
    -		@Override
    -		public int compareTo(final Namespace o) {
    -			if (!namespace.equalsIgnoreCase(o.getName())) {
    +        @Override
    +        public int compareTo(final Namespace o) {
    +            if (!namespace.equalsIgnoreCase(o.getName())) {
                     return namespace.compareTo(o.getName());
                 }
    -			if (!prefix.equalsIgnoreCase(o.getPrefix())) {
    +            if (!prefix.equalsIgnoreCase(o.getPrefix())) {
                     return prefix.compareTo(o.getPrefix());
                 }
    -			return 0;
    -		}
    -
    -		@Override
    -		public String getName() {
    -			return namespace;
    -		}
    -
    -		@Override
    -		public String getPrefix() {
    -			return prefix;
    -		}
    -
    -	}
    -
    -	public class MongoCursorIteration implements
    -			CloseableIteration<Namespace, RyaDAOException> {
    -		private final DBCursor cursor;
    -
    -		public MongoCursorIteration(final DBCursor cursor2) {
    -			this.cursor = cursor2;
    -		}
    -
    -		@Override
    -		public boolean hasNext() throws RyaDAOException {
    -			return cursor.hasNext();
    -		}
    -
    -		@Override
    -		public Namespace next() throws RyaDAOException {
    -			final DBObject ns = cursor.next();
    -			final Map values = ns.toMap();
    -			final String namespace = (String) values.get(NAMESPACE);
    -			final String prefix = (String) values.get(PREFIX);
    -
    -			final Namespace temp =  new NamespaceImplementation(namespace, prefix);
    -			return temp;
    -		}
    -
    -		@Override
    -		public void remove() throws RyaDAOException {
    -			next();
    -		}
    -
    -		@Override
    -		public void close() throws RyaDAOException {
    -			cursor.close();
    -		}
    -
    -	}
    -
    -	private static final String ID = "_id";
    -	private static final String PREFIX = "prefix";
    -	private static final String NAMESPACE = "namespace";
    -	private MongoDBRdfConfiguration conf;
    -	private final DBCollection nsColl;
    -
    -
    -	public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
    -		nsColl = nameSpaceCollection;
    -	}
    -
    -	@Override
    -	public void createIndices(final DBCollection coll){
    -		coll.createIndex(PREFIX);
    -		coll.createIndex(NAMESPACE);
    -	}
    -
    -
    -	@Override
    -	public void setConf(final MongoDBRdfConfiguration paramC) {
    -		this.conf = paramC;
    -	}
    -
    -	@Override
    -	public MongoDBRdfConfiguration getConf() {
    -		// TODO Auto-generated method stub
    -		return conf;
    -	}
    -
    -	@Override
    -	public void addNamespace(final String prefix, final String namespace)
    -			throws RyaDAOException {
    -		final String id = prefix;
    -		byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
    -		try {
    -			final MessageDigest digest = MessageDigest.getInstance("SHA-1");
    -			bytes = digest.digest(bytes);
    -		} catch (final NoSuchAlgorithmException e) {
    -			// TODO Auto-generated catch block
    -			e.printStackTrace();
    -		}
    -		final BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))
    -		.append(PREFIX, prefix)
    -	    .append(NAMESPACE, namespace);
    -		nsColl.insert(doc);
    -
    -	}
    -
    -	@Override
    -	public String getNamespace(final String prefix) throws RyaDAOException {
    +            return 0;
    +        }
    +
    +        @Override
    +        public String getName() {
    +            return namespace;
    +        }
    +
    +        @Override
    +        public String getPrefix() {
    +            return prefix;
    +        }
    +
    +    }
    +
    +    public class MongoCursorIteration implements
    +    CloseableIteration<Namespace, RyaDAOException> {
    +        private final DBCursor cursor;
    +
    +        public MongoCursorIteration(final DBCursor cursor2) {
    +            this.cursor = cursor2;
    +        }
    +
    +        @Override
    +        public boolean hasNext() throws RyaDAOException {
    +            return cursor.hasNext();
    +        }
    +
    +        @Override
    +        public Namespace next() throws RyaDAOException {
    +            final DBObject ns = cursor.next();
    +            final Map values = ns.toMap();
    +            final String namespace = (String) values.get(NAMESPACE);
    +            final String prefix = (String) values.get(PREFIX);
    +
    +            final Namespace temp =  new NamespaceImplementation(namespace, prefix);
    +            return temp;
    +        }
    +
    +        @Override
    +        public void remove() throws RyaDAOException {
    +            next();
    +        }
    +
    +        @Override
    +        public void close() throws RyaDAOException {
    +            cursor.close();
    +        }
    +
    +    }
    +
    +    private static final String ID = "_id";
    +    private static final String PREFIX = "prefix";
    +    private static final String NAMESPACE = "namespace";
    +    private StatefulMongoDBRdfConfiguration conf;
    +    private final DBCollection nsColl;
    +
    +
    +    public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
    +        nsColl = nameSpaceCollection;
    +    }
    +
    +    @Override
    +    public void createIndices(final DBCollection coll){
    +        coll.createIndex(PREFIX);
    +        coll.createIndex(NAMESPACE);
    +    }
    +
    +    @Override
    +    public void setConf(final StatefulMongoDBRdfConfiguration conf) {
    +        this.conf = conf;
    +    }
    +
    +    @Override
    +    public StatefulMongoDBRdfConfiguration getConf() {
    +        return conf;
    +    }
    +
    +    @Override
    +    public void addNamespace(final String prefix, final String namespace)
    +            throws RyaDAOException {
    +        final String id = prefix;
    +        byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
    +        try {
    +            final MessageDigest digest = MessageDigest.getInstance("SHA-1");
    +            bytes = digest.digest(bytes);
    +        } catch (final NoSuchAlgorithmException e) {
    +            // TODO Auto-generated catch block
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159477148
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java ---
    @@ -92,184 +93,162 @@ public MongoDBRdfConfiguration clone() {
             return new MongoDBRdfConfiguration(this);
         }
     
    -    public Authorizations getAuthorizations() {
    -        final String[] auths = getAuths();
    -        if (auths == null || auths.length == 0) {
    -            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    -        }
    -        return new Authorizations(auths);
    -    }
    -
         /**
    -     * @return {@code true} if each statement added to the batch writer should
    -     * be flushed and written right away to the datastore. {@code false} if the
    -     * statements should be queued and written to the datastore when the queue
    -     * is full or after enough time has passed without a write.<p>
    -     * Defaults to {@code true} if nothing is specified.
    +     * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
          */
    -    public boolean flushEachUpdate(){
    -        return getBoolean(CONF_FLUSH_EACH_UPDATE, true);
    +    public void setUseMock(final boolean useMock) {
    +        this.setBoolean(USE_MOCK_MONGO, useMock);
         }
     
         /**
    -     * Sets the {@link #CONF_FLUSH_EACH_UPDATE} property of the configuration.
    -     * @param flush {@code true} if each statement added to the batch writer
    -     * should be flushed and written right away to the datastore. {@code false}
    -     * if the statements should be queued and written to the datastore when the
    -     * queue is full or after enough time has passed without a write.
    +     * Indicates whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @return {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
    --- End diff --
    
    javadoc doesn't like {@true}


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512993
  
    --- Diff: extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java ---
    @@ -36,7 +36,7 @@
     import org.openrdf.model.ValueFactory;
     import org.openrdf.model.impl.ValueFactoryImpl;
     
    -public class GeoTemporalProviderTest extends GeoTemporalTestBase {
    +public class GeoTemporalProviderTest extends GeoTemporalTestUtils {
    --- End diff --
    
    Nope, fixed.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159313589
  
    --- Diff: extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java ---
    @@ -33,15 +32,15 @@
     	/**
     	 * initialize after setting configuration.
     	 */
    -    public void init();  
    +    @Override
    +	public void init();  
    --- End diff --
    
    spacing got thrown off


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159308896
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java ---
    @@ -37,147 +37,145 @@
     
     public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
     
    -	public class NamespaceImplementation implements Namespace {
    +    public class NamespaceImplementation implements Namespace {
     
    -		private final String namespace;
    -		private final String prefix;
    +        private final String namespace;
    +        private final String prefix;
     
    -		public NamespaceImplementation(final String namespace, final String prefix) {
    -			this.namespace = namespace;
    -			this.prefix = prefix;
    -		}
    +        public NamespaceImplementation(final String namespace, final String prefix) {
    +            this.namespace = namespace;
    +            this.prefix = prefix;
    +        }
     
    -		@Override
    -		public int compareTo(final Namespace o) {
    -			if (!namespace.equalsIgnoreCase(o.getName())) {
    +        @Override
    +        public int compareTo(final Namespace o) {
    +            if (!namespace.equalsIgnoreCase(o.getName())) {
                     return namespace.compareTo(o.getName());
                 }
    -			if (!prefix.equalsIgnoreCase(o.getPrefix())) {
    +            if (!prefix.equalsIgnoreCase(o.getPrefix())) {
                     return prefix.compareTo(o.getPrefix());
                 }
    -			return 0;
    -		}
    -
    -		@Override
    -		public String getName() {
    -			return namespace;
    -		}
    -
    -		@Override
    -		public String getPrefix() {
    -			return prefix;
    -		}
    -
    -	}
    -
    -	public class MongoCursorIteration implements
    -			CloseableIteration<Namespace, RyaDAOException> {
    -		private final DBCursor cursor;
    -
    -		public MongoCursorIteration(final DBCursor cursor2) {
    -			this.cursor = cursor2;
    -		}
    -
    -		@Override
    -		public boolean hasNext() throws RyaDAOException {
    -			return cursor.hasNext();
    -		}
    -
    -		@Override
    -		public Namespace next() throws RyaDAOException {
    -			final DBObject ns = cursor.next();
    -			final Map values = ns.toMap();
    -			final String namespace = (String) values.get(NAMESPACE);
    -			final String prefix = (String) values.get(PREFIX);
    -
    -			final Namespace temp =  new NamespaceImplementation(namespace, prefix);
    -			return temp;
    -		}
    -
    -		@Override
    -		public void remove() throws RyaDAOException {
    -			next();
    -		}
    -
    -		@Override
    -		public void close() throws RyaDAOException {
    -			cursor.close();
    -		}
    -
    -	}
    -
    -	private static final String ID = "_id";
    -	private static final String PREFIX = "prefix";
    -	private static final String NAMESPACE = "namespace";
    -	private MongoDBRdfConfiguration conf;
    -	private final DBCollection nsColl;
    -
    -
    -	public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
    -		nsColl = nameSpaceCollection;
    -	}
    -
    -	@Override
    -	public void createIndices(final DBCollection coll){
    -		coll.createIndex(PREFIX);
    -		coll.createIndex(NAMESPACE);
    -	}
    -
    -
    -	@Override
    -	public void setConf(final MongoDBRdfConfiguration paramC) {
    -		this.conf = paramC;
    -	}
    -
    -	@Override
    -	public MongoDBRdfConfiguration getConf() {
    -		// TODO Auto-generated method stub
    -		return conf;
    -	}
    -
    -	@Override
    -	public void addNamespace(final String prefix, final String namespace)
    -			throws RyaDAOException {
    -		final String id = prefix;
    -		byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
    -		try {
    -			final MessageDigest digest = MessageDigest.getInstance("SHA-1");
    -			bytes = digest.digest(bytes);
    -		} catch (final NoSuchAlgorithmException e) {
    -			// TODO Auto-generated catch block
    -			e.printStackTrace();
    -		}
    -		final BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))
    -		.append(PREFIX, prefix)
    -	    .append(NAMESPACE, namespace);
    -		nsColl.insert(doc);
    -
    -	}
    -
    -	@Override
    -	public String getNamespace(final String prefix) throws RyaDAOException {
    +            return 0;
    +        }
    +
    +        @Override
    +        public String getName() {
    +            return namespace;
    +        }
    +
    +        @Override
    +        public String getPrefix() {
    +            return prefix;
    +        }
    +
    +    }
    +
    +    public class MongoCursorIteration implements
    +    CloseableIteration<Namespace, RyaDAOException> {
    +        private final DBCursor cursor;
    +
    +        public MongoCursorIteration(final DBCursor cursor2) {
    +            this.cursor = cursor2;
    +        }
    +
    +        @Override
    +        public boolean hasNext() throws RyaDAOException {
    +            return cursor.hasNext();
    +        }
    +
    +        @Override
    +        public Namespace next() throws RyaDAOException {
    +            final DBObject ns = cursor.next();
    +            final Map values = ns.toMap();
    +            final String namespace = (String) values.get(NAMESPACE);
    +            final String prefix = (String) values.get(PREFIX);
    +
    +            final Namespace temp =  new NamespaceImplementation(namespace, prefix);
    +            return temp;
    +        }
    +
    +        @Override
    +        public void remove() throws RyaDAOException {
    +            next();
    +        }
    +
    +        @Override
    +        public void close() throws RyaDAOException {
    +            cursor.close();
    +        }
    +
    +    }
    +
    +    private static final String ID = "_id";
    +    private static final String PREFIX = "prefix";
    +    private static final String NAMESPACE = "namespace";
    +    private StatefulMongoDBRdfConfiguration conf;
    +    private final DBCollection nsColl;
    +
    +
    +    public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
    +        nsColl = nameSpaceCollection;
    +    }
    +
    +    @Override
    +    public void createIndices(final DBCollection coll){
    +        coll.createIndex(PREFIX);
    +        coll.createIndex(NAMESPACE);
    +    }
    +
    +    @Override
    +    public void setConf(final StatefulMongoDBRdfConfiguration conf) {
    +        this.conf = conf;
    +    }
    +
    +    @Override
    +    public StatefulMongoDBRdfConfiguration getConf() {
    +        return conf;
    +    }
    +
    +    @Override
    +    public void addNamespace(final String prefix, final String namespace)
    +            throws RyaDAOException {
    +        final String id = prefix;
    +        byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
    +        try {
    +            final MessageDigest digest = MessageDigest.getInstance("SHA-1");
    +            bytes = digest.digest(bytes);
    +        } catch (final NoSuchAlgorithmException e) {
    +            // TODO Auto-generated catch block
    --- End diff --
    
    remove


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-rya/pull/256


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512045
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java ---
    @@ -92,184 +93,162 @@ public MongoDBRdfConfiguration clone() {
             return new MongoDBRdfConfiguration(this);
         }
     
    -    public Authorizations getAuthorizations() {
    -        final String[] auths = getAuths();
    -        if (auths == null || auths.length == 0) {
    -            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    -        }
    -        return new Authorizations(auths);
    -    }
    -
         /**
    -     * @return {@code true} if each statement added to the batch writer should
    -     * be flushed and written right away to the datastore. {@code false} if the
    -     * statements should be queued and written to the datastore when the queue
    -     * is full or after enough time has passed without a write.<p>
    -     * Defaults to {@code true} if nothing is specified.
    +     * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
    +     * or if it should connect to a MongoDB Server that is running somewhere.
    +     *
    +     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159484266
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java ---
    @@ -39,44 +35,16 @@
      */
     @DefaultAnnotation(NonNull.class)
     public class MongoEntityIndexer extends BaseEntityIndexer {
    -    private MongoClient client;
    -
    -    @Override
    -    public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoEntityStorage(client, ryaInstanceName);
    -    }
    -
    -    @Override
    -    public TypeStorage getTypeStorage(final Configuration conf) {
    -        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
    -        if (client == null) {
    -            if(mongoConf.getMongoClient() != null) {
    -                client = mongoConf.getMongoClient();
    -            } else {
    -                client = MongoConnectorFactory.getMongoClient(conf);
    -            }
    -        }
    -        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
    -        return new MongoTypeStorage(client, ryaInstanceName);
    -    }
    -
     
         @Override
    -    public void init() {
    -        //nothing to init.
    +    public EntityStorage getEntityStorage() throws EntityStorageException {
    +    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
    --- End diff --
    
    remove tab character
      


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159510828
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java ---
    @@ -503,8 +526,11 @@ public void testVisibility() throws RyaDAOException, MongoException, IOException
          * in the collection. {@code false} otherwise.
          * @throws RyaDAOException
          */
    -    private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
    -        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +    private boolean testVisibilityStatement(
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159511563
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java ---
    @@ -120,7 +120,7 @@ public B setMongoDBName(String name) {
          * Sets MongoDB Collection prefix. This parameter must be set to connect to
          * an instance of MongoDB and will default to "rya_" is no value is
          * specified.
    -     * 
    +     *
          * @param name - name of Collection to connect to
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya issue #256: RYA-414

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/incubator-rya/pull/256
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/589/



---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159311393
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java ---
    @@ -33,27 +34,43 @@
      */
     public class MongoTestBase {
     
    -    private static MongoClient mongoClient = null;
    -    protected static MongoDBRdfConfiguration conf;
    +    private MongoClient mongoClient = null;
    +    protected StatefulMongoDBRdfConfiguration conf;
     
         @Before
         public void setupTest() throws Exception {
    -        conf = new MongoDBRdfConfiguration( new Configuration() );
    +        // Setup the configuration that will be used within the test.
    +        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
             conf.setBoolean("sc.useMongo", true);
             conf.setTablePrefix("test_");
             conf.setMongoDBName("testDB");
    -        mongoClient = EmbeddedMongoSingleton.getInstance();
    -        conf.setMongoClient(mongoClient);
    -    }
    +        conf.setMongoHostname(EmbeddedMongoSingleton.getMongodConfig().net().getServerAddress().getHostAddress());
    +        conf.setMongoPort(Integer.toString(EmbeddedMongoSingleton.getMongodConfig().net().getPort()));
    +
    +        // Let tests update the configuration.
    +        updateConfiguration(conf);
    +
    +        // Create the stateful configuration object.
    +        mongoClient = EmbeddedMongoSingleton.getNewMongoClient();
    +        final List<MongoSecondaryIndex> indexers = conf.getInstances("ac.additional.indexers", MongoSecondaryIndex.class);
    +        this.conf = new StatefulMongoDBRdfConfiguration(conf, mongoClient, indexers);
     
    -    @After
    -    public void cleanupTest() {
    -        // Remove any DBs that were created by the test.
    +        // Remove any DBs that were created by previous tests.
             for(final String dbName : mongoClient.listDatabaseNames()) {
                 mongoClient.dropDatabase(dbName);
             }
         }
     
    +    /**
    +     * Override this method if you would like to augment the configuration object that
    +     * will be used to initialize indexers and create the mongo client prior to running a test.
    +     *
    +     * @param conf - The configuration object that may be updated. (not null)
    +     */
    +    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
    --- End diff --
    
    Do any classes not override this? (I haven't looked to verify)  If most or all classes end up overriding this then this should be changed to an abstract method in an abstract class.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159475679
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java ---
    @@ -0,0 +1,102 @@
    +/**
    + * 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.mongodb;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
    + * {@link Configuration} objects.
    + * </p>
    --- End diff --
    
    should be &lt;p&gt;
      


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159509027
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java ---
    @@ -33,27 +34,43 @@
      */
     public class MongoTestBase {
     
    -    private static MongoClient mongoClient = null;
    -    protected static MongoDBRdfConfiguration conf;
    +    private MongoClient mongoClient = null;
    +    protected StatefulMongoDBRdfConfiguration conf;
     
         @Before
         public void setupTest() throws Exception {
    -        conf = new MongoDBRdfConfiguration( new Configuration() );
    +        // Setup the configuration that will be used within the test.
    +        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
             conf.setBoolean("sc.useMongo", true);
             conf.setTablePrefix("test_");
             conf.setMongoDBName("testDB");
    -        mongoClient = EmbeddedMongoSingleton.getInstance();
    -        conf.setMongoClient(mongoClient);
    -    }
    +        conf.setMongoHostname(EmbeddedMongoSingleton.getMongodConfig().net().getServerAddress().getHostAddress());
    +        conf.setMongoPort(Integer.toString(EmbeddedMongoSingleton.getMongodConfig().net().getPort()));
    +
    +        // Let tests update the configuration.
    +        updateConfiguration(conf);
    +
    +        // Create the stateful configuration object.
    +        mongoClient = EmbeddedMongoSingleton.getNewMongoClient();
    +        final List<MongoSecondaryIndex> indexers = conf.getInstances("ac.additional.indexers", MongoSecondaryIndex.class);
    +        this.conf = new StatefulMongoDBRdfConfiguration(conf, mongoClient, indexers);
     
    -    @After
    -    public void cleanupTest() {
    -        // Remove any DBs that were created by the test.
    +        // Remove any DBs that were created by previous tests.
             for(final String dbName : mongoClient.listDatabaseNames()) {
                 mongoClient.dropDatabase(dbName);
             }
         }
     
    +    /**
    +     * Override this method if you would like to augment the configuration object that
    +     * will be used to initialize indexers and create the mongo client prior to running a test.
    +     *
    +     * @param conf - The configuration object that may be updated. (not null)
    +     */
    +    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
    --- End diff --
    
    Yes. Most of them.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by ejwhite922 <gi...@git.apache.org>.
Github user ejwhite922 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159309681
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java ---
    @@ -503,8 +526,11 @@ public void testVisibility() throws RyaDAOException, MongoException, IOException
          * in the collection. {@code false} otherwise.
          * @throws RyaDAOException
          */
    -    private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
    -        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +    private boolean testVisibilityStatement(
    --- End diff --
    
    Update javadocs with new @param


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159511304
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java ---
    @@ -64,12 +60,13 @@
                     throw new RuntimeException(e);
                 }
                 return (RyaQueryEngine<C>) new AccumuloRyaQueryEngine(conn, aConf);
    -        } else if(conf instanceof MongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
    -            MongoClient client = MongoConnectorFactory.getMongoClient(conf);
    -            return (RyaQueryEngine<C>) new MongoDBQueryEngine((MongoDBRdfConfiguration) conf, client);
    +        } else if(conf instanceof StatefulMongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159512745
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java ---
    @@ -0,0 +1,102 @@
    +/**
    + * 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.mongodb;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
    + * {@link Configuration} objects.
    + * </p>
    + * HACK:
    + * This class is part of a hack to get around how Rya uses reflection to initialize indexers, optimizers, etc.
    --- End diff --
    
    There isn't really a correct solution at the moment. The problem is outlined in that documentation, so I think that's enough.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by kchilton2 <gi...@git.apache.org>.
Github user kchilton2 commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159513721
  
    --- Diff: extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java ---
    @@ -1,64 +0,0 @@
    -/**
    --- End diff --
    
    Renamed it to MongoITBase and updated the names of things that extend it.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159466471
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java ---
    @@ -0,0 +1,102 @@
    +/**
    + * 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.mongodb;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
    + * {@link Configuration} objects.
    + * </p>
    + * HACK:
    + * This class is part of a hack to get around how Rya uses reflection to initialize indexers, optimizers, etc.
    --- End diff --
    
    If this is a hack, can you document what the correct solution is, or reference a JIRA that outlines what needs to change in Rya so this hack can be removed.


---

[GitHub] incubator-rya pull request #256: RYA-414

Posted by jdasch <gi...@git.apache.org>.
Github user jdasch commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/256#discussion_r159466978
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java ---
    @@ -79,19 +79,27 @@ private int findRandomOpenPortOnAllLocalInterfaces() throws IOException {
     
         /**
          * Creates a new Mongo connection.
    -     * 
    +     *
    --- End diff --
    
    add @return


---