You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by amihalik <gi...@git.apache.org> on 2017/08/14 13:59:11 UTC

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

GitHub user amihalik opened a pull request:

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

    RYA-337 Adding batch queries to MongoDB. Closes #204

    ## Description
    
    Added a batch query mechanism to MongoDB DAO and simplified MongoDBQueryEngine
    
    ### Tests
    
    Ran unit tests 
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-337)
    
    ### Checklist
    - [ ] Code Review
    - [ ] Squash Commits
    
    #### People To Review
    @isper3at @pujav65 @meiercaleb 

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

    $ git pull https://github.com/amihalik/incubator-rya RYA-337

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

    https://github.com/apache/incubator-rya/pull/204.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 #204
    
----
commit d19ed3c7fb4ca71bc3c75e1aafc77c4b89c76270
Author: Aaron Mihalik <mi...@alum.mit.edu>
Date:   2017-08-08T15:17:37Z

    RYA-337 Adding batch queries to MongoDB. Closes #204
    
    Additionally, simplifying MongoDBQueryEngine

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r133983577
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -140,72 +121,35 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> batchQuery(
                 final Collection<RyaStatement> stmts, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -        try {
    -            for (final RyaStatement stmt : stmts) {
    -                queries.add( strategy.getQuery(stmt));
    -             }
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
     
    -            // TODO not sure what to do about regex ranges?
    -            final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(getCollection(conf), queries,
    -                    strategy, configuration.getAuthorizations());
    -
    -            if (maxResults != null) {
    -                iterator.setMaxResults(maxResults);
    -            }
    -            return iterator;
    -        } catch (final Exception e) {
    -            throw new RyaDAOException(e);
    +        for (final RyaStatement stmt : stmts) {
    +            queries.put(stmt, new MapBindingSet());
             }
     
    +        return new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), conf));
         }
    +    
         @Override
         public CloseableIterable<RyaStatement> query(final RyaQuery ryaQuery)
                 throws RyaDAOException {
    -            final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -            try {
    -                queries.add( strategy.getQuery(ryaQuery));
    -
    -                // TODO not sure what to do about regex ranges?
    -                // TODO this is gross
    -            final RyaStatementCursorIterable iterator = new RyaStatementCursorIterable(
    -                    new NonCloseableRyaStatementCursorIterator(new RyaStatementCursorIterator(getCollection(getConf()),
    -                            queries, strategy, configuration.getAuthorizations())));
    -
    -                return iterator;
    -            } catch (final Exception e) {
    -                throw new RyaDAOException(e);
    -            }
    +        return query(new BatchRyaQuery(Collections.singleton(ryaQuery.getQuery())));
         }
    +
         @Override
         public CloseableIterable<RyaStatement> query(final BatchRyaQuery batchRyaQuery)
                 throws RyaDAOException {
    -         try {
    -             final Set<DBObject> queries = new HashSet<DBObject>();
    -            for (final RyaStatement statement : batchRyaQuery.getQueries()){
    -                queries.add( strategy.getQuery(statement));
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r133983248
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -111,25 +97,20 @@ public MongoDBRdfConfiguration getConf() {
             if (conf == null) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132979641
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -91,47 +96,81 @@ private boolean currentBindingSetIteratorIsValid() {
         }
     
         private void findNextResult() {
    -        if (!currentResultCursorIsValid()) {
    -            findNextValidResultCursor();
    +        if (!currentBatchQueryResultCursorIsValid()) {
    +            submitBatchQuery();
             }
    -        if (currentResultCursorIsValid()) {
    +        
    +        if (currentBatchQueryResultCursorIsValid()) {
                 // convert to Rya Statement
    -            final Document queryResult = resultsIterator.next();
    +            final Document queryResult = batchQueryResultsIterator.next();
                 final DBObject dbo = (DBObject) JSON.parse(queryResult.toJson());
    -            currentStatement = strategy.deserializeDBObject(dbo);
    -            currentBindingSetIterator = currentBindingSetCollection.iterator();
    +            currentResultStatement = strategy.deserializeDBObject(dbo);
    +            
    +            // Find all of the queries in the executed RangeMap that this result matches
    +            // and collect all of those binding sets
    +            Set<BindingSet> bsList = new HashSet<>();
    +            for (RyaStatement executedQuery : executedRangeMap.keys()) {
    +                if (isResultForQuery(executedQuery, currentResultStatement)) {
    +                    bsList.addAll(executedRangeMap.get(executedQuery));
    +                }
    +            }
    +            currentBindingSetIterator = bsList.iterator();
    +        }
    +        
    +        // Handle case of invalid currentResultStatement or no binding sets returned
    +        if ((currentBindingSetIterator == null || !currentBindingSetIterator.hasNext()) && (currentBatchQueryResultCursorIsValid() || queryIterator.hasNext())) {
    +            findNextResult();
             }
         }
    +    
    +    private static boolean isResultForQuery(RyaStatement query, RyaStatement result) {
    +        return isResult(query.getSubject(), result.getSubject()) &&
    +                isResult(query.getPredicate(), result.getPredicate()) &&
    +                isResult(query.getObject(), result.getObject()) &&
    +                isResult(query.getContext(), result.getContext());
    +    }
    +    
    +    private static boolean isResult(RyaType query, RyaType result) {
    +        return (query == null) || query.equals(result);
    +    }
     
    -    private void findNextValidResultCursor() {
    -        while (queryIterator.hasNext()){
    -            final DBObject currentQuery = queryIterator.next();
    -            currentBindingSetCollection = rangeMap.get(currentQuery);
    -            // Executing redact aggregation to only return documents the user
    -            // has access to.
    -            final List<Document> pipeline = new ArrayList<>();
    -            pipeline.add(new Document("$match", currentQuery));
    -            pipeline.addAll(AggregationUtil.createRedactPipeline(auths));
    -            log.debug(pipeline);
    -
    -            final AggregateIterable<Document> aggIter = coll.aggregate(pipeline);
    -            aggIter.batchSize(1000);
    -            resultsIterator = aggIter.iterator();
    -            if (resultsIterator.hasNext()) {
    -                break;
    -            }
    +    private void submitBatchQuery() {
    +        int count = 0;
    +        executedRangeMap.clear();
    +        final List<Document> pipeline = new ArrayList<>();
    +        final List<DBObject> match = new ArrayList<>();
    +
    +        while (queryIterator.hasNext() && count < QUERY_BATCH_SIZE){
    +            count++;
    +            RyaStatement query = queryIterator.next();
    +            executedRangeMap.putAll(query, rangeMap.get(query));
    +            final DBObject currentQuery = strategy.getQuery(query);
    +            match.add(currentQuery);
             }
    -    }
     
    -    private boolean currentResultCursorIsValid() {
    -        return (resultsIterator != null) && resultsIterator.hasNext();
    +        if (match.size() > 1) {
    +            pipeline.add(new Document("$match", new Document("$or", match)));
    --- End diff --
    
    Did you compare the performance of $or with $in? Seems like $or is used to see if a field value satisfies at least one of (possibly) many general logical expressions, while $in checks to see if a value is in some sort of indexed collection.  My guess is that the added generality of $or would make it less performant for batch queries than $in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132989696
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -86,22 +84,10 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> query(
                 final RyaStatement stmt, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -        final DBObject query = strategy.getQuery(stmt);
    -        queries.add(query);
    -        final MongoDatabase db = mongoClient.getDatabase(conf.getMongoDBName());
    -        final MongoCollection<Document> collection = db.getCollection(conf.getTriplesCollectionName());
    -        final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(collection, queries, strategy,
    -                conf.getAuthorizations());
    -
    -        if (maxResults != null) {
    -            iterator.setMaxResults(maxResults);
    -        }
    -        return iterator;
    +        Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
    --- End diff --
    
    Check that the RyaStatement and Config are not null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132990271
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -111,25 +97,20 @@ public MongoDBRdfConfiguration getConf() {
             if (conf == null) {
    --- End diff --
    
    Check that stmts is not null


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132990694
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -140,72 +121,35 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> batchQuery(
                 final Collection<RyaStatement> stmts, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -        try {
    -            for (final RyaStatement stmt : stmts) {
    -                queries.add( strategy.getQuery(stmt));
    -             }
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
     
    -            // TODO not sure what to do about regex ranges?
    -            final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(getCollection(conf), queries,
    -                    strategy, configuration.getAuthorizations());
    -
    -            if (maxResults != null) {
    -                iterator.setMaxResults(maxResults);
    -            }
    -            return iterator;
    -        } catch (final Exception e) {
    -            throw new RyaDAOException(e);
    +        for (final RyaStatement stmt : stmts) {
    +            queries.put(stmt, new MapBindingSet());
             }
     
    +        return new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), conf));
         }
    +    
         @Override
         public CloseableIterable<RyaStatement> query(final RyaQuery ryaQuery)
                 throws RyaDAOException {
    -            final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -            try {
    -                queries.add( strategy.getQuery(ryaQuery));
    -
    -                // TODO not sure what to do about regex ranges?
    -                // TODO this is gross
    -            final RyaStatementCursorIterable iterator = new RyaStatementCursorIterable(
    -                    new NonCloseableRyaStatementCursorIterator(new RyaStatementCursorIterator(getCollection(getConf()),
    -                            queries, strategy, configuration.getAuthorizations())));
    -
    -                return iterator;
    -            } catch (final Exception e) {
    -                throw new RyaDAOException(e);
    -            }
    +        return query(new BatchRyaQuery(Collections.singleton(ryaQuery.getQuery())));
         }
    +
         @Override
         public CloseableIterable<RyaStatement> query(final BatchRyaQuery batchRyaQuery)
                 throws RyaDAOException {
    -         try {
    -             final Set<DBObject> queries = new HashSet<DBObject>();
    -            for (final RyaStatement statement : batchRyaQuery.getQueries()){
    -                queries.add( strategy.getQuery(statement));
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
    --- End diff --
    
    Null check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r133983396
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -140,72 +121,35 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> batchQuery(
                 final Collection<RyaStatement> stmts, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -        try {
    -            for (final RyaStatement stmt : stmts) {
    -                queries.add( strategy.getQuery(stmt));
    -             }
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
     
    -            // TODO not sure what to do about regex ranges?
    -            final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(getCollection(conf), queries,
    -                    strategy, configuration.getAuthorizations());
    -
    -            if (maxResults != null) {
    -                iterator.setMaxResults(maxResults);
    -            }
    -            return iterator;
    -        } catch (final Exception e) {
    -            throw new RyaDAOException(e);
    +        for (final RyaStatement stmt : stmts) {
    +            queries.put(stmt, new MapBindingSet());
             }
     
    +        return new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), conf));
         }
    +    
         @Override
         public CloseableIterable<RyaStatement> query(final RyaQuery ryaQuery)
                 throws RyaDAOException {
    -            final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -            try {
    -                queries.add( strategy.getQuery(ryaQuery));
    -
    -                // TODO not sure what to do about regex ranges?
    -                // TODO this is gross
    -            final RyaStatementCursorIterable iterator = new RyaStatementCursorIterable(
    -                    new NonCloseableRyaStatementCursorIterator(new RyaStatementCursorIterator(getCollection(getConf()),
    -                            queries, strategy, configuration.getAuthorizations())));
    -
    -                return iterator;
    -            } catch (final Exception e) {
    -                throw new RyaDAOException(e);
    -            }
    +        return query(new BatchRyaQuery(Collections.singleton(ryaQuery.getQuery())));
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya issue #204: RYA-337 Adding batch queries to MongoDB. Closes #2...

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

    https://github.com/apache/incubator-rya/pull/204
  
    
    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/386/<h2>Failed Tests: <span class='status-failure'>1</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.indexing.example' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/386/org.apache.rya$rya.indexing.example/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.indexing.example</a>: <span class='status-failure'>1</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/386/org.apache.rya$rya.indexing.example/testReport/(root)/ExamplesTest/MongoRyaDirectExampleTest/'><strong>ExamplesTest.MongoRyaDirectExampleTest</strong></a></li></ul>



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya issue #204: RYA-337 Adding batch queries to MongoDB. Closes #2...

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

    https://github.com/apache/incubator-rya/pull/204
  
    
    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/387/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132990636
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -140,72 +121,35 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> batchQuery(
                 final Collection<RyaStatement> stmts, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -        try {
    -            for (final RyaStatement stmt : stmts) {
    -                queries.add( strategy.getQuery(stmt));
    -             }
    +        final Map<RyaStatement, BindingSet> queries = new HashMap<>();
     
    -            // TODO not sure what to do about regex ranges?
    -            final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(getCollection(conf), queries,
    -                    strategy, configuration.getAuthorizations());
    -
    -            if (maxResults != null) {
    -                iterator.setMaxResults(maxResults);
    -            }
    -            return iterator;
    -        } catch (final Exception e) {
    -            throw new RyaDAOException(e);
    +        for (final RyaStatement stmt : stmts) {
    +            queries.put(stmt, new MapBindingSet());
             }
     
    +        return new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), conf));
         }
    +    
         @Override
         public CloseableIterable<RyaStatement> query(final RyaQuery ryaQuery)
                 throws RyaDAOException {
    -            final Set<DBObject> queries = new HashSet<DBObject>();
    -
    -            try {
    -                queries.add( strategy.getQuery(ryaQuery));
    -
    -                // TODO not sure what to do about regex ranges?
    -                // TODO this is gross
    -            final RyaStatementCursorIterable iterator = new RyaStatementCursorIterable(
    -                    new NonCloseableRyaStatementCursorIterator(new RyaStatementCursorIterator(getCollection(getConf()),
    -                            queries, strategy, configuration.getAuthorizations())));
    -
    -                return iterator;
    -            } catch (final Exception e) {
    -                throw new RyaDAOException(e);
    -            }
    +        return query(new BatchRyaQuery(Collections.singleton(ryaQuery.getQuery())));
    --- End diff --
    
    Null check


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya issue #204: RYA-337 Adding batch queries to MongoDB. Closes #2...

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

    https://github.com/apache/incubator-rya/pull/204
  
    
    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/410/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r133984224
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -44,20 +48,21 @@
     
     public class RyaStatementBindingSetCursorIterator implements CloseableIteration<Entry<RyaStatement, BindingSet>, RyaDAOException> {
         private static final Logger log = Logger.getLogger(RyaStatementBindingSetCursorIterator.class);
    +    
    +    private static final int QUERY_BATCH_SIZE = 50;
    --- End diff --
    
    Empirical testing... To support the "show me the first 10" or "first 100" type queries.  I noticed that if we left this at 1000, then the "show me the first 10" or "first 100" type queries were very slow.  However, the "show me all" (where all = 30k), we equally as quick using 50 or 1000.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r133983122
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -86,22 +84,10 @@ public MongoDBRdfConfiguration getConf() {
         public CloseableIteration<RyaStatement, RyaDAOException> query(
                 final RyaStatement stmt, MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        if (conf == null) {
    -            conf = configuration;
    -        }
    -        final Long maxResults = conf.getLimit();
    -        final Set<DBObject> queries = new HashSet<DBObject>();
    -        final DBObject query = strategy.getQuery(stmt);
    -        queries.add(query);
    -        final MongoDatabase db = mongoClient.getDatabase(conf.getMongoDBName());
    -        final MongoCollection<Document> collection = db.getCollection(conf.getTriplesCollectionName());
    -        final RyaStatementCursorIterator iterator = new RyaStatementCursorIterator(collection, queries, strategy,
    -                conf.getAuthorizations());
    -
    -        if (maxResults != null) {
    -            iterator.setMaxResults(maxResults);
    -        }
    -        return iterator;
    +        Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-rya pull request #204: RYA-337 Adding batch queries to MongoDB. Cl...

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

    https://github.com/apache/incubator-rya/pull/204#discussion_r132980017
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -44,20 +48,21 @@
     
     public class RyaStatementBindingSetCursorIterator implements CloseableIteration<Entry<RyaStatement, BindingSet>, RyaDAOException> {
         private static final Logger log = Logger.getLogger(RyaStatementBindingSetCursorIterator.class);
    +    
    +    private static final int QUERY_BATCH_SIZE = 50;
    --- End diff --
    
    Any particular reason you went with 50 here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---