You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by jessehatfield <gi...@git.apache.org> on 2017/12/21 22:45:49 UTC

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

GitHub user jessehatfield opened a pull request:

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

    RYA-416 Optionally invoke aggregation pipeline to execute MongoDB queries

    [WIP] Provides tools for converting some or all of a SPARQL query to an aggregation pipeline.
    <!--
    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.
    -->
    ## Description
    >What Changed?
    [Brief Description of what changed]
    
    ### Tests
    >Coverage?
    
    [Description of what tests were written]
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-NUMBER)
    
    ### Checklist
    - [ ] Code Review
    - [ ] Squash Commits
    
    #### People To Reivew
    [Add those who should review this]


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

    $ git pull https://github.com/jessehatfield/incubator-rya RYA-416-aggregation-pipeline

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

    https://github.com/apache/incubator-rya/pull/254.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 #254
    
----
commit 3eed2f92cc534768a6627a8c348eeef2c820c0e9
Author: Jesse Hatfield <je...@...>
Date:   2017-12-21T22:32:47Z

    A new query node type to represent a MongoDB aggregation pipeline whose results can be converted to binding sets, and tools for optionally transforming some SPARQL expressions into such a node.

----


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159311056
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    --- End diff --
    
    The checks return the checked object, so this could be 
    ```
    import static java.util.Objects.requireNonNull;
    ...
    this.collection = requireNonNull(collection)
    ```
    Comment applies through out this review.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949139
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/SparqlToPipelineTransformVisitor.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Arrays;
    +
    +import org.apache.rya.mongodb.MongoConnectorFactory;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.bson.Document;
    +import org.openrdf.query.algebra.Distinct;
    +import org.openrdf.query.algebra.Extension;
    +import org.openrdf.query.algebra.Filter;
    +import org.openrdf.query.algebra.Join;
    +import org.openrdf.query.algebra.MultiProjection;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.Reduced;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
    +
    +public class SparqlToPipelineTransformVisitor extends QueryModelVisitorBase<Exception> {
    --- End diff --
    
    Added.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r160798766
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    --- End diff --
    
    I agree that we're inconsistent within our code, but I think our pom files only ever explicitly pull in slf4j. I think log4j is being used accidentally when it is brought in as a transitive dependency.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159311494
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    --- End diff --
    
    Objects.hash( /* stuff here */ );


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r160800939
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    +        lookahead();
    +        nextSolution = null;
    +    }
    +
    +    @Override
    +    public void close() throws QueryEvaluationException {
    +        cursor.close();
    +    }
    +
    +    private QueryBindingSet docToBindingSet(Document result) {
    +        QueryBindingSet bindingSet = new QueryBindingSet(bindings);
    +        Document valueSet = result.get(AggregationPipelineQueryNode.VALUES, Document.class);
    +        Document typeSet = result.get(AggregationPipelineQueryNode.TYPES, Document.class);
    +        if (valueSet != null) {
    +            for (Map.Entry<String, Object> entry : valueSet.entrySet()) {
    +                String fieldName = entry.getKey();
    +                String valueString = entry.getValue().toString();
    +                String typeString = typeSet == null ? null : typeSet.getString(fieldName);
    +                String varName = varToOriginalName.getOrDefault(fieldName, fieldName);
    +                Value varValue;
    +                if (typeString == null || typeString.equals(XMLSchema.ANYURI.stringValue())) {
    +                    varValue = VF.createURI(valueString);
    +                }
    +                else {
    +                    varValue = VF.createLiteral(valueString, VF.createURI(typeString));
    +                }
    +                Binding existingBinding = bindingSet.getBinding(varName);
    +                // If this variable is not already bound, add it.
    +                if (existingBinding == null) {
    +                    bindingSet.addBinding(varName, varValue);
    +                }
    +                // If it's bound to something else, the solutions are incompatible.
    +                else if (!existingBinding.getValue().equals(varValue)) {
    +                    return null;
    --- End diff --
    
    Oh I see how that's being used now. Thanks for the explanation.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159948988
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949217
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/SparqlToPipelineTransformVisitor.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Arrays;
    +
    +import org.apache.rya.mongodb.MongoConnectorFactory;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.bson.Document;
    +import org.openrdf.query.algebra.Distinct;
    +import org.openrdf.query.algebra.Extension;
    +import org.openrdf.query.algebra.Filter;
    +import org.openrdf.query.algebra.Join;
    +import org.openrdf.query.algebra.MultiProjection;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.Reduced;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
    +
    +public class SparqlToPipelineTransformVisitor extends QueryModelVisitorBase<Exception> {
    +    private MongoCollection<Document> inputCollection;
    +
    +    public SparqlToPipelineTransformVisitor(MongoCollection<Document> inputCollection) {
    +        this.inputCollection = inputCollection;
    --- End diff --
    
    Shouldn't be, check added.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159332716
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    --- End diff --
    
    This should throw an unsupported exception. Remove means you're removing the previously returned object from the underlying collection. This doesn't do that.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159943074
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    +        lookahead();
    +        nextSolution = null;
    +    }
    +
    +    @Override
    +    public void close() throws QueryEvaluationException {
    +        cursor.close();
    +    }
    +
    +    private QueryBindingSet docToBindingSet(Document result) {
    +        QueryBindingSet bindingSet = new QueryBindingSet(bindings);
    +        Document valueSet = result.get(AggregationPipelineQueryNode.VALUES, Document.class);
    +        Document typeSet = result.get(AggregationPipelineQueryNode.TYPES, Document.class);
    +        if (valueSet != null) {
    +            for (Map.Entry<String, Object> entry : valueSet.entrySet()) {
    +                String fieldName = entry.getKey();
    +                String valueString = entry.getValue().toString();
    +                String typeString = typeSet == null ? null : typeSet.getString(fieldName);
    +                String varName = varToOriginalName.getOrDefault(fieldName, fieldName);
    +                Value varValue;
    +                if (typeString == null || typeString.equals(XMLSchema.ANYURI.stringValue())) {
    +                    varValue = VF.createURI(valueString);
    +                }
    +                else {
    +                    varValue = VF.createLiteral(valueString, VF.createURI(typeString));
    +                }
    +                Binding existingBinding = bindingSet.getBinding(varName);
    +                // If this variable is not already bound, add it.
    +                if (existingBinding == null) {
    +                    bindingSet.addBinding(varName, varValue);
    +                }
    +                // If it's bound to something else, the solutions are incompatible.
    +                else if (!existingBinding.getValue().equals(varValue)) {
    +                    return null;
    --- End diff --
    
    This method is only called internally, by lookahead(). It returns null when the next solution returned by the pipeline is incompatible with any bindings that were passed into the constructor. (The constructor may have received a non-empty BindingSet if the pipeline query is being evaluated in the context of a specific partial solution.) The lookahead method will use this to loop through pipeline results until it finds one that is compatible with the bindings, or until the pipeline yields no more results. The null value should never make its way to the caller. I didn't use an exception here because it isn't really an exceptional case in that context: we should expect some of the solutions to the pipeline subquery to not match the specific candidate solution.
    
    Skipping over non-matching results client-side might be inefficient, and we could implement this instead by adding a "$match" step for the candidate solution onto the pipeline. But this wouldn't solve the main inefficiency which is that either way we're executing the pipeline subquery for each candidate solution. So the approach I've taken here is just to do something that satisfies the API correctly, and hope that we can structure queries to avoid the inherent inefficiency. (For example, if we had `Join(Join(SP1, SP2), Join(SP3, SP4))`, the visitor as currently implemented would turn that into a join of two pipeline nodes and have to pass binding sets from one into the other, which will be slow regardless of how that's done. But if it's instead parsed as `Join(Join(Join(SP1, SP2), SP3), SP4))` then we can translate the whole thing into one pipeline query, which should be faster.)
    
    I added a test for this iterator which includes the case of a provided BindingSet and incompatible partial solutions, verifying that the expected (non-null) results are returned.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159306671
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    --- End diff --
    
    We're using slf4j instead of log4j.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159950342
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    --- End diff --
    
    Changed to "$max", weird oversight on my part. Also added a specific check to the appropriate test to convince myself it works.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159950589
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    +        // 1. Determine shared variables and new variables
    +        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
    +        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
    +        sharedVars.retainAll(assuredBindingNames);
    +        // 2. Join on one shared variable
    +        String joinKey =  sharedVars.pollFirst();
    +        String collectionName = collection.getNamespace().getCollectionName();
    +        Bson join;
    +        if (joinKey == null) {
    +            return false;
    +        }
    +        else {
    +            join = Aggregates.lookup(collectionName,
    +                    HASHES + "." + joinKey,
    +                    spMap.hashField(joinKey),
    +                    JOINED_TRIPLE);
    +        }
    +        pipeline.add(join);
    +        // 3. Unwind the joined triples so each document represents a binding
    +        //   set (solution) from the base branch and a triple that may match.
    +        pipeline.add(Aggregates.unwind("$" + JOINED_TRIPLE));
    +        // 4. (Optional) If there are any shared variables that weren't used as
    +        //   the join key, project all existing fields plus a new field that
    +        //   tests the equality of those shared variables.
    +        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
    +        if (!sharedVars.isEmpty()) {
    +            List<Bson> eqTests = new LinkedList<>();
    +            for (String varName : sharedVars) {
    +                String oldField = valueFieldExpr(varName);
    +                String newField = joinFieldExpr(spMap.valueField(varName));
    +                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
    +                eqTests.add(eqTest);
    +            }
    +            Bson eqProjectOpts = Projections.fields(
    +                    Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
    +                    Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
    +            pipeline.add(Aggregates.project(eqProjectOpts));
    +            matchOpts.put(FIELDS_MATCH, true);
    +        }
    +        // 5. Filter for solutions whose triples match the joined statement
    +        //  pattern, and, if applicable, whose additional shared variables
    +        //  match the current solution.
    +        pipeline.add(Aggregates.match(matchOpts));
    +        // 5. Project the results to include variables from the new SP (with
    --- End diff --
    
    Fixed.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159306746
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.Dataset;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.algebra.evaluation.QueryOptimizer;
    +
    +public class AggregationPipelineQueryOptimizer implements QueryOptimizer, Configurable {
    +    private Configuration conf;
    +    private Logger logger = Logger.getLogger(getClass());
    +
    +    @Override
    +    public void optimize(TupleExpr tupleExpr, Dataset dataset, BindingSet bindings) {
    +        if (conf instanceof MongoDBRdfConfiguration) {
    --- End diff --
    
    This will likely change into a StatefulMongoDBRdfConfiguration in 414 (which is a MongoDBRdfConfiguration). Is anything about your optimizer stateful? If so, you don't want to make more than one instance of it, so it should be shared within that config.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159307900
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/SparqlToPipelineTransformVisitor.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Arrays;
    +
    +import org.apache.rya.mongodb.MongoConnectorFactory;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.bson.Document;
    +import org.openrdf.query.algebra.Distinct;
    +import org.openrdf.query.algebra.Extension;
    +import org.openrdf.query.algebra.Filter;
    +import org.openrdf.query.algebra.Join;
    +import org.openrdf.query.algebra.MultiProjection;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.Reduced;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
    +
    +public class SparqlToPipelineTransformVisitor extends QueryModelVisitorBase<Exception> {
    --- End diff --
    
    Docs.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159332382
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    --- End diff --
    
    This should be removed. All of your methods perform lookahead(). In general, a constructor shouldn't be invoking methods on something that communicates on a network since it may take a long time to complete when things aren't working properly.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159745332
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    --- End diff --
    
    why not use mongo's $max?
    Add $max to Aggregates?


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159950551
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    --- End diff --
    
    Changed.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r160799419
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.Dataset;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.algebra.evaluation.QueryOptimizer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * MongoDB-specific query optimizer that replaces part or all of a SPARQL query
    + * tree with a MongoDB aggregation pipeline.
    + * <p>
    + * Transforms query trees using {@link SparqlToPipelineTransformVisitor}. If
    + * possible, this visitor will replace portions of the query tree, or the entire
    + * query, with an equivalent aggregation pipeline (contained in an
    + * {@link AggregationPipelineQueryNode}), thereby allowing query logic to be
    + * evaluated by the MongoDB server rather than by the client.
    + */
    +public class AggregationPipelineQueryOptimizer implements QueryOptimizer, Configurable {
    +    private Configuration conf;
    +    private Logger logger = LoggerFactory.getLogger(getClass());
    +
    +    @Override
    +    public void optimize(TupleExpr tupleExpr, Dataset dataset, BindingSet bindings) {
    +        if (conf instanceof StatefulMongoDBRdfConfiguration) {
    +            StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            SparqlToPipelineTransformVisitor pipelineVisitor = new SparqlToPipelineTransformVisitor(mongoConf);
    +            try {
    +                tupleExpr.visit(pipelineVisitor);
    +            } catch (Exception e) {
    +                logger.error("Error attempting to transform query using the aggregation pipeline", e);
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void setConf(Configuration conf) {
    +        this.conf = conf;
    --- End diff --
    
    This should probably throw an IllegalArgumentException if it is provided a conf object that is not of type StatefulMongoDBRdfConfiguration because the code will not work if that is the case. Throwing an exception make it easier to hunt down that bug.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159746172
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    +        // 1. Determine shared variables and new variables
    +        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
    +        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
    +        sharedVars.retainAll(assuredBindingNames);
    +        // 2. Join on one shared variable
    +        String joinKey =  sharedVars.pollFirst();
    +        String collectionName = collection.getNamespace().getCollectionName();
    +        Bson join;
    +        if (joinKey == null) {
    +            return false;
    +        }
    +        else {
    +            join = Aggregates.lookup(collectionName,
    +                    HASHES + "." + joinKey,
    +                    spMap.hashField(joinKey),
    +                    JOINED_TRIPLE);
    +        }
    +        pipeline.add(join);
    +        // 3. Unwind the joined triples so each document represents a binding
    +        //   set (solution) from the base branch and a triple that may match.
    +        pipeline.add(Aggregates.unwind("$" + JOINED_TRIPLE));
    +        // 4. (Optional) If there are any shared variables that weren't used as
    +        //   the join key, project all existing fields plus a new field that
    +        //   tests the equality of those shared variables.
    +        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
    +        if (!sharedVars.isEmpty()) {
    +            List<Bson> eqTests = new LinkedList<>();
    +            for (String varName : sharedVars) {
    +                String oldField = valueFieldExpr(varName);
    +                String newField = joinFieldExpr(spMap.valueField(varName));
    +                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
    +                eqTests.add(eqTest);
    +            }
    +            Bson eqProjectOpts = Projections.fields(
    +                    Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
    +                    Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
    +            pipeline.add(Aggregates.project(eqProjectOpts));
    +            matchOpts.put(FIELDS_MATCH, true);
    +        }
    +        // 5. Filter for solutions whose triples match the joined statement
    +        //  pattern, and, if applicable, whose additional shared variables
    +        //  match the current solution.
    +        pipeline.add(Aggregates.match(matchOpts));
    +        // 5. Project the results to include variables from the new SP (with
    +        // appropriate renaming) and variables referenced only in the base
    +        // pipeline (with previous names).
    +        Bson finalProjectOpts = new StatementVarMapping(sp, varToOriginalName)
    +                .getProjectExpression(assuredBindingNames,
    +                        str -> joinFieldExpr(str));
    +        assuredBindingNames.addAll(spMap.varNames());
    +        bindingNames.addAll(spMap.varNames());
    +        pipeline.add(Aggregates.project(finalProjectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL projection or multi-projection operation to the pipeline.
    +     * The number of documents produced by the pipeline after this operation
    +     * will be the number of documents entering this stage (the number of
    +     * intermediate results) multiplied by the number of
    +     * {@link ProjectionElemList}s supplied here.
    +     * @param projections One or more projections, i.e. mappings from the result
    +     *  at this stage of the query into a set of variables.
    +     * @return true if the projection(s) were added to the pipeline.
    +     */
    +    public boolean project(Iterable<ProjectionElemList> projections) {
    +        if (projections == null || !projections.iterator().hasNext()) {
    +            return false;
    +        }
    +        List<Bson> projectOpts = new LinkedList<>();
    +        Set<String> bindingNamesUnion = new HashSet<>();
    +        Set<String> bindingNamesIntersection = null;
    +        for (ProjectionElemList projection : projections) {
    +            Document valueDoc = new Document();
    +            Document hashDoc = new Document();
    +            Document typeDoc = new Document();
    +            Set<String> projectionBindingNames = new HashSet<>();
    +            for (ProjectionElem elem : projection.getElements()) {
    +                String to = elem.getTargetName();
    +                // If the 'to' name is invalid, replace it internally
    +                if (!isValidFieldName(to)) {
    +                    to = replace(to);
    +                }
    +                String from = elem.getSourceName();
    +                // If the 'from' name is invalid, use the internal substitute
    +                if (varToOriginalName.containsValue(from)) {
    +                    from = varToOriginalName.inverse().get(from);
    +                }
    +                projectionBindingNames.add(to);
    +                if (to.equals(from)) {
    +                    valueDoc.append(to, 1);
    +                    hashDoc.append(to, 1);
    +                    typeDoc.append(to, 1);
    +                }
    +                else {
    +                    valueDoc.append(to, valueFieldExpr(from));
    +                    hashDoc.append(to, hashFieldExpr(from));
    +                    typeDoc.append(to, typeFieldExpr(from));
    +                }
    +            }
    +            bindingNamesUnion.addAll(projectionBindingNames);
    +            if (bindingNamesIntersection == null) {
    +                bindingNamesIntersection = new HashSet<>(projectionBindingNames);
    +            }
    +            else {
    +                bindingNamesIntersection.retainAll(projectionBindingNames);
    +            }
    +            projectOpts.add(new Document()
    +                    .append(VALUES, valueDoc)
    +                    .append(HASHES, hashDoc)
    +                    .append(TYPES, typeDoc)
    +                    .append(LEVEL, "$" + LEVEL)
    +                    .append(TIMESTAMP, "$" + TIMESTAMP));
    +        }
    +        if (projectOpts.size() == 1) {
    +            pipeline.add(Aggregates.project(projectOpts.get(0)));
    +        }
    +        else {
    +            String listKey = "PROJECTIONS";
    +            Bson projectIndividual = Projections.fields(
    +                    Projections.computed(VALUES, "$" + listKey + "." + VALUES),
    +                    Projections.computed(HASHES, "$" + listKey + "." + HASHES),
    +                    Projections.computed(TYPES, "$" + listKey + "." + TYPES),
    +                    Projections.include(LEVEL),
    +                    Projections.include(TIMESTAMP));
    +            pipeline.add(Aggregates.project(Projections.computed(listKey, projectOpts)));
    +            pipeline.add(Aggregates.unwind("$" + listKey));
    +            pipeline.add(Aggregates.project(projectIndividual));
    +        }
    +        assuredBindingNames.clear();
    +        bindingNames.clear();
    +        assuredBindingNames.addAll(bindingNamesIntersection);
    +        bindingNames.addAll(bindingNamesUnion);
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL extension to the pipeline, if possible. An extension adds
    +     * some number of variables to the result. Adds a "$project" step to the
    +     * pipeline, but differs from the SPARQL project operation in that
    +     * 1) pre-existing variables are always kept, and 2) values of new variables
    +     * are defined by expressions, which may be more complex than simply
    +     * variable names. Not all expressions are supported. If unsupported
    +     * expression types are used in the extension, the pipeline will remain
    +     * unchanged and this method will return false.
    +     * @param extensionElements A list of new variables and their expressions
    +     * @return True if the extension was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean extend(Iterable<ExtensionElem> extensionElements) {
    +        List<Bson> valueFields = new LinkedList<>();
    +        List<Bson> hashFields = new LinkedList<>();
    +        List<Bson> typeFields = new LinkedList<>();
    +        for (String varName : bindingNames) {
    +            valueFields.add(Projections.include(varName));
    +            hashFields.add(Projections.include(varName));
    +            typeFields.add(Projections.include(varName));
    +        }
    +        Set<String> newVarNames = new HashSet<>();
    +        for (ExtensionElem elem : extensionElements) {
    +            String name = elem.getName();
    +            if (!isValidFieldName(name)) {
    +                // If the field name is invalid, replace it internally
    +                name = replace(name);
    +            }
    +            // We can only handle certain kinds of value expressions; return
    +            // failure for any others.
    +            ValueExpr expr = elem.getExpr();
    +            final Object valueField;
    +            final Object hashField;
    +            final Object typeField;
    +            if (expr instanceof Var) {
    +                String varName = ((Var) expr).getName();
    +                valueField = "$" + varName;
    +                hashField = "$" + varName;
    +                typeField = "$" + varName;
    +            }
    +            else if (expr instanceof ValueConstant) {
    +                Value val = ((ValueConstant) expr).getValue();
    +                valueField = new Document("$literal", val.stringValue());
    +                hashField = new Document("$literal", SimpleMongoDBStorageStrategy.hash(val.stringValue()));
    +                if (val instanceof Literal) {
    +                    typeField = new Document("$literal", ((Literal) val).getDatatype().stringValue());
    +                }
    +                else {
    +                    typeField = null;
    +                }
    +            }
    +            else {
    +                // if not understood, return failure
    +                return false;
    +            }
    +            valueFields.add(Projections.computed(name, valueField));
    +            hashFields.add(Projections.computed(name, hashField));
    +            if (typeField != null) {
    +                typeFields.add(Projections.computed(name, typeField));
    +            }
    +            newVarNames.add(name);
    +        }
    +        assuredBindingNames.addAll(newVarNames);
    +        bindingNames.addAll(newVarNames);
    +        Bson projectOpts = Projections.fields(
    +                Projections.computed(VALUES, Projections.fields(valueFields)),
    +                Projections.computed(HASHES, Projections.fields(hashFields)),
    +                Projections.computed(TYPES, Projections.fields(typeFields)),
    +                Projections.include(LEVEL),
    +                Projections.include(TIMESTAMP));
    +        pipeline.add(Aggregates.project(projectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL filter to the pipeline, if possible. A filter eliminates
    +     * results that don't satisfy a given condition. Not all conditional
    +     * expressions are supported. If unsupported expressions are used in the
    +     * filter, the pipeline will remain unchanged and this method will return
    +     * false.
    +     * @param condition The filter condition
    +     * @return True if the filter was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean filter(ValueExpr condition) {
    +        if (condition instanceof Compare) {
    +            Compare compare = (Compare) condition;
    +            Compare.CompareOp operator = compare.getOperator();
    +            Object leftArg = valueFieldExpr(compare.getLeftArg());
    +            Object rightArg = valueFieldExpr(compare.getRightArg());
    +            if (leftArg == null || rightArg == null) {
    +                // unsupported value expression, can't convert filter
    +                return false;
    +            }
    +            final String opFunc;
    +            switch (operator) {
    +            case EQ:
    +                opFunc = "$eq";
    +                break;
    +            case NE:
    +                opFunc = "$ne";
    +                break;
    +            case LT:
    +                opFunc = "$lt";
    +                break;
    +            case LE:
    +                opFunc = "$le";
    +                break;
    +            case GT:
    +                opFunc = "$gt";
    +                break;
    +            case GE:
    +                opFunc = "$ge";
    +                break;
    +            default:
    +                // unrecognized comparison operator, can't convert filter
    +                return false;
    +            }
    +            Document compareDoc = new Document(opFunc, Arrays.asList(leftArg, rightArg));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.computed("FILTER", compareDoc),
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            pipeline.add(Aggregates.match(new Document("FILTER", true)));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            return true;
    +        }
    +        return false;
    +    }
    +
    +    /**
    +     * Add a $group step to filter out redundant solutions.
    +     * @return True if the distinct operation was successfully appended.
    +     */
    +    public boolean distinct() {
    --- End diff --
    
    there is a mongo function for finding distinct values over fields


---


[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159932088
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    +        // 1. Determine shared variables and new variables
    +        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
    +        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
    +        sharedVars.retainAll(assuredBindingNames);
    +        // 2. Join on one shared variable
    +        String joinKey =  sharedVars.pollFirst();
    +        String collectionName = collection.getNamespace().getCollectionName();
    +        Bson join;
    +        if (joinKey == null) {
    +            return false;
    +        }
    +        else {
    +            join = Aggregates.lookup(collectionName,
    +                    HASHES + "." + joinKey,
    +                    spMap.hashField(joinKey),
    +                    JOINED_TRIPLE);
    +        }
    +        pipeline.add(join);
    +        // 3. Unwind the joined triples so each document represents a binding
    +        //   set (solution) from the base branch and a triple that may match.
    +        pipeline.add(Aggregates.unwind("$" + JOINED_TRIPLE));
    +        // 4. (Optional) If there are any shared variables that weren't used as
    +        //   the join key, project all existing fields plus a new field that
    +        //   tests the equality of those shared variables.
    +        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
    +        if (!sharedVars.isEmpty()) {
    +            List<Bson> eqTests = new LinkedList<>();
    +            for (String varName : sharedVars) {
    +                String oldField = valueFieldExpr(varName);
    +                String newField = joinFieldExpr(spMap.valueField(varName));
    +                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
    +                eqTests.add(eqTest);
    +            }
    +            Bson eqProjectOpts = Projections.fields(
    +                    Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
    +                    Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
    +            pipeline.add(Aggregates.project(eqProjectOpts));
    +            matchOpts.put(FIELDS_MATCH, true);
    +        }
    +        // 5. Filter for solutions whose triples match the joined statement
    +        //  pattern, and, if applicable, whose additional shared variables
    +        //  match the current solution.
    +        pipeline.add(Aggregates.match(matchOpts));
    +        // 5. Project the results to include variables from the new SP (with
    +        // appropriate renaming) and variables referenced only in the base
    +        // pipeline (with previous names).
    +        Bson finalProjectOpts = new StatementVarMapping(sp, varToOriginalName)
    +                .getProjectExpression(assuredBindingNames,
    +                        str -> joinFieldExpr(str));
    +        assuredBindingNames.addAll(spMap.varNames());
    +        bindingNames.addAll(spMap.varNames());
    +        pipeline.add(Aggregates.project(finalProjectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL projection or multi-projection operation to the pipeline.
    +     * The number of documents produced by the pipeline after this operation
    +     * will be the number of documents entering this stage (the number of
    +     * intermediate results) multiplied by the number of
    +     * {@link ProjectionElemList}s supplied here.
    +     * @param projections One or more projections, i.e. mappings from the result
    +     *  at this stage of the query into a set of variables.
    +     * @return true if the projection(s) were added to the pipeline.
    +     */
    +    public boolean project(Iterable<ProjectionElemList> projections) {
    +        if (projections == null || !projections.iterator().hasNext()) {
    +            return false;
    +        }
    +        List<Bson> projectOpts = new LinkedList<>();
    +        Set<String> bindingNamesUnion = new HashSet<>();
    +        Set<String> bindingNamesIntersection = null;
    +        for (ProjectionElemList projection : projections) {
    +            Document valueDoc = new Document();
    +            Document hashDoc = new Document();
    +            Document typeDoc = new Document();
    +            Set<String> projectionBindingNames = new HashSet<>();
    +            for (ProjectionElem elem : projection.getElements()) {
    +                String to = elem.getTargetName();
    +                // If the 'to' name is invalid, replace it internally
    +                if (!isValidFieldName(to)) {
    +                    to = replace(to);
    +                }
    +                String from = elem.getSourceName();
    +                // If the 'from' name is invalid, use the internal substitute
    +                if (varToOriginalName.containsValue(from)) {
    +                    from = varToOriginalName.inverse().get(from);
    +                }
    +                projectionBindingNames.add(to);
    +                if (to.equals(from)) {
    +                    valueDoc.append(to, 1);
    +                    hashDoc.append(to, 1);
    +                    typeDoc.append(to, 1);
    +                }
    +                else {
    +                    valueDoc.append(to, valueFieldExpr(from));
    +                    hashDoc.append(to, hashFieldExpr(from));
    +                    typeDoc.append(to, typeFieldExpr(from));
    +                }
    +            }
    +            bindingNamesUnion.addAll(projectionBindingNames);
    +            if (bindingNamesIntersection == null) {
    +                bindingNamesIntersection = new HashSet<>(projectionBindingNames);
    +            }
    +            else {
    +                bindingNamesIntersection.retainAll(projectionBindingNames);
    +            }
    +            projectOpts.add(new Document()
    +                    .append(VALUES, valueDoc)
    +                    .append(HASHES, hashDoc)
    +                    .append(TYPES, typeDoc)
    +                    .append(LEVEL, "$" + LEVEL)
    +                    .append(TIMESTAMP, "$" + TIMESTAMP));
    +        }
    +        if (projectOpts.size() == 1) {
    +            pipeline.add(Aggregates.project(projectOpts.get(0)));
    +        }
    +        else {
    +            String listKey = "PROJECTIONS";
    +            Bson projectIndividual = Projections.fields(
    +                    Projections.computed(VALUES, "$" + listKey + "." + VALUES),
    +                    Projections.computed(HASHES, "$" + listKey + "." + HASHES),
    +                    Projections.computed(TYPES, "$" + listKey + "." + TYPES),
    +                    Projections.include(LEVEL),
    +                    Projections.include(TIMESTAMP));
    +            pipeline.add(Aggregates.project(Projections.computed(listKey, projectOpts)));
    +            pipeline.add(Aggregates.unwind("$" + listKey));
    +            pipeline.add(Aggregates.project(projectIndividual));
    +        }
    +        assuredBindingNames.clear();
    +        bindingNames.clear();
    +        assuredBindingNames.addAll(bindingNamesIntersection);
    +        bindingNames.addAll(bindingNamesUnion);
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL extension to the pipeline, if possible. An extension adds
    +     * some number of variables to the result. Adds a "$project" step to the
    +     * pipeline, but differs from the SPARQL project operation in that
    +     * 1) pre-existing variables are always kept, and 2) values of new variables
    +     * are defined by expressions, which may be more complex than simply
    +     * variable names. Not all expressions are supported. If unsupported
    +     * expression types are used in the extension, the pipeline will remain
    +     * unchanged and this method will return false.
    +     * @param extensionElements A list of new variables and their expressions
    +     * @return True if the extension was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean extend(Iterable<ExtensionElem> extensionElements) {
    +        List<Bson> valueFields = new LinkedList<>();
    +        List<Bson> hashFields = new LinkedList<>();
    +        List<Bson> typeFields = new LinkedList<>();
    +        for (String varName : bindingNames) {
    +            valueFields.add(Projections.include(varName));
    +            hashFields.add(Projections.include(varName));
    +            typeFields.add(Projections.include(varName));
    +        }
    +        Set<String> newVarNames = new HashSet<>();
    +        for (ExtensionElem elem : extensionElements) {
    +            String name = elem.getName();
    +            if (!isValidFieldName(name)) {
    +                // If the field name is invalid, replace it internally
    +                name = replace(name);
    +            }
    +            // We can only handle certain kinds of value expressions; return
    +            // failure for any others.
    +            ValueExpr expr = elem.getExpr();
    +            final Object valueField;
    +            final Object hashField;
    +            final Object typeField;
    +            if (expr instanceof Var) {
    +                String varName = ((Var) expr).getName();
    +                valueField = "$" + varName;
    +                hashField = "$" + varName;
    +                typeField = "$" + varName;
    +            }
    +            else if (expr instanceof ValueConstant) {
    +                Value val = ((ValueConstant) expr).getValue();
    +                valueField = new Document("$literal", val.stringValue());
    +                hashField = new Document("$literal", SimpleMongoDBStorageStrategy.hash(val.stringValue()));
    +                if (val instanceof Literal) {
    +                    typeField = new Document("$literal", ((Literal) val).getDatatype().stringValue());
    +                }
    +                else {
    +                    typeField = null;
    +                }
    +            }
    +            else {
    +                // if not understood, return failure
    +                return false;
    +            }
    +            valueFields.add(Projections.computed(name, valueField));
    +            hashFields.add(Projections.computed(name, hashField));
    +            if (typeField != null) {
    +                typeFields.add(Projections.computed(name, typeField));
    +            }
    +            newVarNames.add(name);
    +        }
    +        assuredBindingNames.addAll(newVarNames);
    +        bindingNames.addAll(newVarNames);
    +        Bson projectOpts = Projections.fields(
    +                Projections.computed(VALUES, Projections.fields(valueFields)),
    +                Projections.computed(HASHES, Projections.fields(hashFields)),
    +                Projections.computed(TYPES, Projections.fields(typeFields)),
    +                Projections.include(LEVEL),
    +                Projections.include(TIMESTAMP));
    +        pipeline.add(Aggregates.project(projectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL filter to the pipeline, if possible. A filter eliminates
    +     * results that don't satisfy a given condition. Not all conditional
    +     * expressions are supported. If unsupported expressions are used in the
    +     * filter, the pipeline will remain unchanged and this method will return
    +     * false.
    +     * @param condition The filter condition
    +     * @return True if the filter was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean filter(ValueExpr condition) {
    +        if (condition instanceof Compare) {
    +            Compare compare = (Compare) condition;
    +            Compare.CompareOp operator = compare.getOperator();
    +            Object leftArg = valueFieldExpr(compare.getLeftArg());
    +            Object rightArg = valueFieldExpr(compare.getRightArg());
    +            if (leftArg == null || rightArg == null) {
    +                // unsupported value expression, can't convert filter
    +                return false;
    +            }
    +            final String opFunc;
    +            switch (operator) {
    +            case EQ:
    +                opFunc = "$eq";
    +                break;
    +            case NE:
    +                opFunc = "$ne";
    +                break;
    +            case LT:
    +                opFunc = "$lt";
    +                break;
    +            case LE:
    +                opFunc = "$le";
    +                break;
    +            case GT:
    +                opFunc = "$gt";
    +                break;
    +            case GE:
    +                opFunc = "$ge";
    +                break;
    +            default:
    +                // unrecognized comparison operator, can't convert filter
    +                return false;
    +            }
    +            Document compareDoc = new Document(opFunc, Arrays.asList(leftArg, rightArg));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.computed("FILTER", compareDoc),
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            pipeline.add(Aggregates.match(new Document("FILTER", true)));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            return true;
    +        }
    +        return false;
    +    }
    +
    +    /**
    +     * Add a $group step to filter out redundant solutions.
    +     * @return True if the distinct operation was successfully appended.
    +     */
    +    public boolean distinct() {
    --- End diff --
    
    Can you elaborate? If there's a cleaner way to accomplish something like a SPARQL "DISTINCT" operation, I haven't found it yet -- keep in mind we don't want a field containing a list of distinct values, but rather a separate document returned for each distinct value. This solution is based on [this example](https://docs.mongodb.com/manual/reference/operator/aggregation/group/#retrieve-distinct-values) which uses $group to retrieve distinct results, except that the key is constructed from the variable hashes and all the fields are kept (using "$first" to choose one value for each since the values within the group should be identical).


---


[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159950434
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    --- End diff --
    
    Updated.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159333249
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    +        lookahead();
    +        nextSolution = null;
    +    }
    +
    +    @Override
    +    public void close() throws QueryEvaluationException {
    +        cursor.close();
    +    }
    +
    +    private QueryBindingSet docToBindingSet(Document result) {
    +        QueryBindingSet bindingSet = new QueryBindingSet(bindings);
    +        Document valueSet = result.get(AggregationPipelineQueryNode.VALUES, Document.class);
    +        Document typeSet = result.get(AggregationPipelineQueryNode.TYPES, Document.class);
    +        if (valueSet != null) {
    +            for (Map.Entry<String, Object> entry : valueSet.entrySet()) {
    +                String fieldName = entry.getKey();
    +                String valueString = entry.getValue().toString();
    +                String typeString = typeSet == null ? null : typeSet.getString(fieldName);
    +                String varName = varToOriginalName.getOrDefault(fieldName, fieldName);
    +                Value varValue;
    +                if (typeString == null || typeString.equals(XMLSchema.ANYURI.stringValue())) {
    +                    varValue = VF.createURI(valueString);
    +                }
    +                else {
    +                    varValue = VF.createLiteral(valueString, VF.createURI(typeString));
    +                }
    +                Binding existingBinding = bindingSet.getBinding(varName);
    +                // If this variable is not already bound, add it.
    +                if (existingBinding == null) {
    +                    bindingSet.addBinding(varName, varValue);
    +                }
    +                // If it's bound to something else, the solutions are incompatible.
    +                else if (!existingBinding.getValue().equals(varValue)) {
    +                    return null;
    --- End diff --
    
    This would cause your Iteration to periodically return null which will likely result in a NPE by whoever is using it since that would be pretty unexpected.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

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


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159925007
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    --- End diff --
    
    Added.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r160808196
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    +        // 1. Determine shared variables and new variables
    +        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
    +        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
    +        sharedVars.retainAll(assuredBindingNames);
    +        // 2. Join on one shared variable
    +        String joinKey =  sharedVars.pollFirst();
    +        String collectionName = collection.getNamespace().getCollectionName();
    +        Bson join;
    +        if (joinKey == null) {
    +            return false;
    +        }
    +        else {
    +            join = Aggregates.lookup(collectionName,
    +                    HASHES + "." + joinKey,
    +                    spMap.hashField(joinKey),
    +                    JOINED_TRIPLE);
    +        }
    +        pipeline.add(join);
    +        // 3. Unwind the joined triples so each document represents a binding
    +        //   set (solution) from the base branch and a triple that may match.
    +        pipeline.add(Aggregates.unwind("$" + JOINED_TRIPLE));
    +        // 4. (Optional) If there are any shared variables that weren't used as
    +        //   the join key, project all existing fields plus a new field that
    +        //   tests the equality of those shared variables.
    +        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
    +        if (!sharedVars.isEmpty()) {
    +            List<Bson> eqTests = new LinkedList<>();
    +            for (String varName : sharedVars) {
    +                String oldField = valueFieldExpr(varName);
    +                String newField = joinFieldExpr(spMap.valueField(varName));
    +                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
    +                eqTests.add(eqTest);
    +            }
    +            Bson eqProjectOpts = Projections.fields(
    +                    Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
    +                    Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
    +            pipeline.add(Aggregates.project(eqProjectOpts));
    +            matchOpts.put(FIELDS_MATCH, true);
    +        }
    +        // 5. Filter for solutions whose triples match the joined statement
    +        //  pattern, and, if applicable, whose additional shared variables
    +        //  match the current solution.
    +        pipeline.add(Aggregates.match(matchOpts));
    +        // 5. Project the results to include variables from the new SP (with
    +        // appropriate renaming) and variables referenced only in the base
    +        // pipeline (with previous names).
    +        Bson finalProjectOpts = new StatementVarMapping(sp, varToOriginalName)
    +                .getProjectExpression(assuredBindingNames,
    +                        str -> joinFieldExpr(str));
    +        assuredBindingNames.addAll(spMap.varNames());
    +        bindingNames.addAll(spMap.varNames());
    +        pipeline.add(Aggregates.project(finalProjectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL projection or multi-projection operation to the pipeline.
    +     * The number of documents produced by the pipeline after this operation
    +     * will be the number of documents entering this stage (the number of
    +     * intermediate results) multiplied by the number of
    +     * {@link ProjectionElemList}s supplied here.
    +     * @param projections One or more projections, i.e. mappings from the result
    +     *  at this stage of the query into a set of variables.
    +     * @return true if the projection(s) were added to the pipeline.
    +     */
    +    public boolean project(Iterable<ProjectionElemList> projections) {
    +        if (projections == null || !projections.iterator().hasNext()) {
    +            return false;
    +        }
    +        List<Bson> projectOpts = new LinkedList<>();
    +        Set<String> bindingNamesUnion = new HashSet<>();
    +        Set<String> bindingNamesIntersection = null;
    +        for (ProjectionElemList projection : projections) {
    +            Document valueDoc = new Document();
    +            Document hashDoc = new Document();
    +            Document typeDoc = new Document();
    +            Set<String> projectionBindingNames = new HashSet<>();
    +            for (ProjectionElem elem : projection.getElements()) {
    +                String to = elem.getTargetName();
    +                // If the 'to' name is invalid, replace it internally
    +                if (!isValidFieldName(to)) {
    +                    to = replace(to);
    +                }
    +                String from = elem.getSourceName();
    +                // If the 'from' name is invalid, use the internal substitute
    +                if (varToOriginalName.containsValue(from)) {
    +                    from = varToOriginalName.inverse().get(from);
    +                }
    +                projectionBindingNames.add(to);
    +                if (to.equals(from)) {
    +                    valueDoc.append(to, 1);
    +                    hashDoc.append(to, 1);
    +                    typeDoc.append(to, 1);
    +                }
    +                else {
    +                    valueDoc.append(to, valueFieldExpr(from));
    +                    hashDoc.append(to, hashFieldExpr(from));
    +                    typeDoc.append(to, typeFieldExpr(from));
    +                }
    +            }
    +            bindingNamesUnion.addAll(projectionBindingNames);
    +            if (bindingNamesIntersection == null) {
    +                bindingNamesIntersection = new HashSet<>(projectionBindingNames);
    +            }
    +            else {
    +                bindingNamesIntersection.retainAll(projectionBindingNames);
    +            }
    +            projectOpts.add(new Document()
    +                    .append(VALUES, valueDoc)
    +                    .append(HASHES, hashDoc)
    +                    .append(TYPES, typeDoc)
    +                    .append(LEVEL, "$" + LEVEL)
    +                    .append(TIMESTAMP, "$" + TIMESTAMP));
    +        }
    +        if (projectOpts.size() == 1) {
    +            pipeline.add(Aggregates.project(projectOpts.get(0)));
    +        }
    +        else {
    +            String listKey = "PROJECTIONS";
    +            Bson projectIndividual = Projections.fields(
    +                    Projections.computed(VALUES, "$" + listKey + "." + VALUES),
    +                    Projections.computed(HASHES, "$" + listKey + "." + HASHES),
    +                    Projections.computed(TYPES, "$" + listKey + "." + TYPES),
    +                    Projections.include(LEVEL),
    +                    Projections.include(TIMESTAMP));
    +            pipeline.add(Aggregates.project(Projections.computed(listKey, projectOpts)));
    +            pipeline.add(Aggregates.unwind("$" + listKey));
    +            pipeline.add(Aggregates.project(projectIndividual));
    +        }
    +        assuredBindingNames.clear();
    +        bindingNames.clear();
    +        assuredBindingNames.addAll(bindingNamesIntersection);
    +        bindingNames.addAll(bindingNamesUnion);
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL extension to the pipeline, if possible. An extension adds
    +     * some number of variables to the result. Adds a "$project" step to the
    +     * pipeline, but differs from the SPARQL project operation in that
    +     * 1) pre-existing variables are always kept, and 2) values of new variables
    +     * are defined by expressions, which may be more complex than simply
    +     * variable names. Not all expressions are supported. If unsupported
    +     * expression types are used in the extension, the pipeline will remain
    +     * unchanged and this method will return false.
    +     * @param extensionElements A list of new variables and their expressions
    +     * @return True if the extension was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean extend(Iterable<ExtensionElem> extensionElements) {
    +        List<Bson> valueFields = new LinkedList<>();
    +        List<Bson> hashFields = new LinkedList<>();
    +        List<Bson> typeFields = new LinkedList<>();
    +        for (String varName : bindingNames) {
    +            valueFields.add(Projections.include(varName));
    +            hashFields.add(Projections.include(varName));
    +            typeFields.add(Projections.include(varName));
    +        }
    +        Set<String> newVarNames = new HashSet<>();
    +        for (ExtensionElem elem : extensionElements) {
    +            String name = elem.getName();
    +            if (!isValidFieldName(name)) {
    +                // If the field name is invalid, replace it internally
    +                name = replace(name);
    +            }
    +            // We can only handle certain kinds of value expressions; return
    +            // failure for any others.
    +            ValueExpr expr = elem.getExpr();
    +            final Object valueField;
    +            final Object hashField;
    +            final Object typeField;
    +            if (expr instanceof Var) {
    +                String varName = ((Var) expr).getName();
    +                valueField = "$" + varName;
    +                hashField = "$" + varName;
    +                typeField = "$" + varName;
    +            }
    +            else if (expr instanceof ValueConstant) {
    +                Value val = ((ValueConstant) expr).getValue();
    +                valueField = new Document("$literal", val.stringValue());
    +                hashField = new Document("$literal", SimpleMongoDBStorageStrategy.hash(val.stringValue()));
    +                if (val instanceof Literal) {
    +                    typeField = new Document("$literal", ((Literal) val).getDatatype().stringValue());
    +                }
    +                else {
    +                    typeField = null;
    +                }
    +            }
    +            else {
    +                // if not understood, return failure
    +                return false;
    +            }
    +            valueFields.add(Projections.computed(name, valueField));
    +            hashFields.add(Projections.computed(name, hashField));
    +            if (typeField != null) {
    +                typeFields.add(Projections.computed(name, typeField));
    +            }
    +            newVarNames.add(name);
    +        }
    +        assuredBindingNames.addAll(newVarNames);
    +        bindingNames.addAll(newVarNames);
    +        Bson projectOpts = Projections.fields(
    +                Projections.computed(VALUES, Projections.fields(valueFields)),
    +                Projections.computed(HASHES, Projections.fields(hashFields)),
    +                Projections.computed(TYPES, Projections.fields(typeFields)),
    +                Projections.include(LEVEL),
    +                Projections.include(TIMESTAMP));
    +        pipeline.add(Aggregates.project(projectOpts));
    +        return true;
    +    }
    +
    +    /**
    +     * Add a SPARQL filter to the pipeline, if possible. A filter eliminates
    +     * results that don't satisfy a given condition. Not all conditional
    +     * expressions are supported. If unsupported expressions are used in the
    +     * filter, the pipeline will remain unchanged and this method will return
    +     * false.
    +     * @param condition The filter condition
    +     * @return True if the filter was successfully converted into a pipeline
    +     *  step, false otherwise.
    +     */
    +    public boolean filter(ValueExpr condition) {
    +        if (condition instanceof Compare) {
    +            Compare compare = (Compare) condition;
    +            Compare.CompareOp operator = compare.getOperator();
    +            Object leftArg = valueFieldExpr(compare.getLeftArg());
    +            Object rightArg = valueFieldExpr(compare.getRightArg());
    +            if (leftArg == null || rightArg == null) {
    +                // unsupported value expression, can't convert filter
    +                return false;
    +            }
    +            final String opFunc;
    +            switch (operator) {
    +            case EQ:
    +                opFunc = "$eq";
    +                break;
    +            case NE:
    +                opFunc = "$ne";
    +                break;
    +            case LT:
    +                opFunc = "$lt";
    +                break;
    +            case LE:
    +                opFunc = "$le";
    +                break;
    +            case GT:
    +                opFunc = "$gt";
    +                break;
    +            case GE:
    +                opFunc = "$ge";
    +                break;
    +            default:
    +                // unrecognized comparison operator, can't convert filter
    +                return false;
    +            }
    +            Document compareDoc = new Document(opFunc, Arrays.asList(leftArg, rightArg));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.computed("FILTER", compareDoc),
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            pipeline.add(Aggregates.match(new Document("FILTER", true)));
    +            pipeline.add(Aggregates.project(Projections.fields(
    +                    Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
    +            return true;
    +        }
    +        return false;
    +    }
    +
    +    /**
    +     * Add a $group step to filter out redundant solutions.
    +     * @return True if the distinct operation was successfully appended.
    +     */
    +    public boolean distinct() {
    --- End diff --
    
    yeah I misunderstood, I though SPARQL DISTINCT was similar to mongo's distinct, or vis versa.  I mean you might be able to use distinct, then do another operation to split up the resulting field into separate documents, but if that's the case this is fine.


---


[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159307607
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -328,4 +328,22 @@ public static boolean doesUserHaveDocumentAccess(final Authorizations authorizat
             }
             return list.toArray(new Object[0]);
         }
    +
    +    /**
    +     * Converts a {@link List} into an array of {@link Object}s.
    --- End diff --
    
    It might be worth mentioning that it flattens lists that are found within inputList into the returned array. What does this do to other data structures though? You'll leave a Set or Map as such in the array? Are other collections just allowed to be in the returned array?


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159333105
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    +        lookahead();
    +        nextSolution = null;
    +    }
    +
    +    @Override
    +    public void close() throws QueryEvaluationException {
    +        cursor.close();
    +    }
    +
    +    private QueryBindingSet docToBindingSet(Document result) {
    +        QueryBindingSet bindingSet = new QueryBindingSet(bindings);
    +        Document valueSet = result.get(AggregationPipelineQueryNode.VALUES, Document.class);
    +        Document typeSet = result.get(AggregationPipelineQueryNode.TYPES, Document.class);
    +        if (valueSet != null) {
    +            for (Map.Entry<String, Object> entry : valueSet.entrySet()) {
    +                String fieldName = entry.getKey();
    +                String valueString = entry.getValue().toString();
    +                String typeString = typeSet == null ? null : typeSet.getString(fieldName);
    +                String varName = varToOriginalName.getOrDefault(fieldName, fieldName);
    +                Value varValue;
    +                if (typeString == null || typeString.equals(XMLSchema.ANYURI.stringValue())) {
    +                    varValue = VF.createURI(valueString);
    +                }
    +                else {
    +                    varValue = VF.createLiteral(valueString, VF.createURI(typeString));
    +                }
    +                Binding existingBinding = bindingSet.getBinding(varName);
    +                // If this variable is not already bound, add it.
    +                if (existingBinding == null) {
    +                    bindingSet.addBinding(varName, varValue);
    +                }
    +                // If it's bound to something else, the solutions are incompatible.
    +                else if (!existingBinding.getValue().equals(varValue)) {
    +                    return null;
    --- End diff --
    
    Should this throw an exception? Why does it return null?


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159924919
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159948871
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    --- End diff --
    
    Changed, though it looks like we're inconsistent on this.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159331857
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    --- End diff --
    
    All of these need to be null checked.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

    https://github.com/apache/incubator-rya/pull/254
  
    
    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/601/<h2>Failed Tests: <span class='status-failure'>3</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.prospector' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/601/org.apache.rya$rya.prospector/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.prospector</a>: <span class='status-failure'>3</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/601/org.apache.rya$rya.prospector/testReport/org.apache.rya.prospector.mr/ProspectorTest/testCount/'><strong>org.apache.rya.prospector.mr.ProspectorTest.testCount</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/601/org.apache.rya$rya.prospector/testReport/org.apache.rya.prospector.service/ProspectorServiceEvalStatsDAOTest/testCount
 /'><strong>org.apache.rya.prospector.service.ProspectorServiceEvalStatsDAOTest.testCount</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/601/org.apache.rya$rya.prospector/testReport/org.apache.rya.prospector.service/ProspectorServiceEvalStatsDAOTest/testNoAuthsCount/'><strong>org.apache.rya.prospector.service.ProspectorServiceEvalStatsDAOTest.testNoAuthsCount</strong></a></li></ul>



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159310844
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    --- End diff --
    
    The documentation of this seems to be out of date since it accepts a collection.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159335982
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    --- End diff --
    
    null check


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949935
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -328,4 +328,22 @@ public static boolean doesUserHaveDocumentAccess(final Authorizations authorizat
             }
             return list.toArray(new Object[0]);
         }
    +
    +    /**
    +     * Converts a {@link List} into an array of {@link Object}s.
    --- End diff --
    
    Removed this method anyway, turned out to be unnecessary. (I had some incorrect way of converting a Document to a DBObject; if done properly, the existing methods are sufficient.)


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

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



---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159948908
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.Dataset;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.algebra.evaluation.QueryOptimizer;
    +
    +public class AggregationPipelineQueryOptimizer implements QueryOptimizer, Configurable {
    --- End diff --
    
    Added.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949072
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    +    }
    +
    +    private void lookahead() {
    +        while (nextSolution == null && cursor.hasNext()) {
    +            nextSolution = docToBindingSet(cursor.next());
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() throws QueryEvaluationException {
    +        lookahead();
    +        return nextSolution != null;
    +    }
    +
    +    @Override
    +    public BindingSet next() throws QueryEvaluationException {
    +        lookahead();
    +        BindingSet solution = nextSolution;
    +        nextSolution = null;
    +        return solution;
    +    }
    +
    +    @Override
    +    public void remove() throws QueryEvaluationException {
    --- End diff --
    
    Agreed, done.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r160811757
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.Dataset;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.algebra.evaluation.QueryOptimizer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * MongoDB-specific query optimizer that replaces part or all of a SPARQL query
    + * tree with a MongoDB aggregation pipeline.
    + * <p>
    + * Transforms query trees using {@link SparqlToPipelineTransformVisitor}. If
    + * possible, this visitor will replace portions of the query tree, or the entire
    + * query, with an equivalent aggregation pipeline (contained in an
    + * {@link AggregationPipelineQueryNode}), thereby allowing query logic to be
    + * evaluated by the MongoDB server rather than by the client.
    + */
    +public class AggregationPipelineQueryOptimizer implements QueryOptimizer, Configurable {
    +    private Configuration conf;
    +    private Logger logger = LoggerFactory.getLogger(getClass());
    +
    +    @Override
    +    public void optimize(TupleExpr tupleExpr, Dataset dataset, BindingSet bindings) {
    +        if (conf instanceof StatefulMongoDBRdfConfiguration) {
    +            StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            SparqlToPipelineTransformVisitor pipelineVisitor = new SparqlToPipelineTransformVisitor(mongoConf);
    +            try {
    +                tupleExpr.visit(pipelineVisitor);
    +            } catch (Exception e) {
    +                logger.error("Error attempting to transform query using the aggregation pipeline", e);
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void setConf(Configuration conf) {
    +        this.conf = conf;
    --- End diff --
    
    Agreed, fixed.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949017
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/PipelineResultIteration.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Map;
    +
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.ValueFactory;
    +import org.openrdf.model.impl.ValueFactoryImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.Binding;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCursor;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * An iterator that converts the documents resulting from an
    + * {@link AggregationPipelineQueryNode} into {@link BindingSet}s.
    + */
    +public class PipelineResultIteration implements CloseableIteration<BindingSet, QueryEvaluationException> {
    +    private static final int BATCH_SIZE = 1000;
    +    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
    +
    +    private final MongoCursor<Document> cursor;
    +    private final Map<String, String> varToOriginalName;
    +    private final BindingSet bindings;
    +    private BindingSet nextSolution = null;
    +
    +    /**
    +     * Constructor.
    +     * @param aggIter Iterator of documents in AggregationPipelineQueryNode's
    +     *  intermediate solution representation.
    +     * @param varToOriginalName A mapping from field names in the pipeline
    +     *  result documents to equivalent variable names in the original query.
    +     *  Where an entry does not exist for a field, the field name and variable
    +     *  name are assumed to be the same.
    +     * @param bindings A partial solution. May be empty.
    +     */
    +    public PipelineResultIteration(AggregateIterable<Document> aggIter,
    +            Map<String, String> varToOriginalName,
    +            BindingSet bindings) {
    +        aggIter.batchSize(BATCH_SIZE);
    +        this.cursor = aggIter.iterator();
    +        this.varToOriginalName = varToOriginalName;
    +        this.bindings = bindings;
    +        lookahead();
    --- End diff --
    
    Removed.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159308088
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/SparqlToPipelineTransformVisitor.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Arrays;
    +
    +import org.apache.rya.mongodb.MongoConnectorFactory;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.bson.Document;
    +import org.openrdf.query.algebra.Distinct;
    +import org.openrdf.query.algebra.Extension;
    +import org.openrdf.query.algebra.Filter;
    +import org.openrdf.query.algebra.Join;
    +import org.openrdf.query.algebra.MultiProjection;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.Reduced;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
    +
    +public class SparqlToPipelineTransformVisitor extends QueryModelVisitorBase<Exception> {
    +    private MongoCollection<Document> inputCollection;
    +
    +    public SparqlToPipelineTransformVisitor(MongoCollection<Document> inputCollection) {
    +        this.inputCollection = inputCollection;
    +    }
    +
    +    public SparqlToPipelineTransformVisitor(MongoDBRdfConfiguration conf) {
    +        MongoClient mongo = MongoConnectorFactory.getMongoClient(conf);
    --- End diff --
    
    MongoConnectorFactory no longer exists after RYA-414. You will receive a stateful configuration object that contains the MongoClient.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159306588
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryOptimizer.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.aggregation;
    +
    +import org.apache.hadoop.conf.Configurable;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.Dataset;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.algebra.evaluation.QueryOptimizer;
    +
    +public class AggregationPipelineQueryOptimizer implements QueryOptimizer, Configurable {
    --- End diff --
    
    Docs.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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

    https://github.com/apache/incubator-rya/pull/254#discussion_r159949267
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -63,6 +63,10 @@
         public static final String STATEMENT_METADATA = "statementMetadata";
         public static final String DOCUMENT_VISIBILITY = "documentVisibility";
     
    +    public static String hash(String value) {
    --- End diff --
    
    Added.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159307946
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/SparqlToPipelineTransformVisitor.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.aggregation;
    +
    +import java.util.Arrays;
    +
    +import org.apache.rya.mongodb.MongoConnectorFactory;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.bson.Document;
    +import org.openrdf.query.algebra.Distinct;
    +import org.openrdf.query.algebra.Extension;
    +import org.openrdf.query.algebra.Filter;
    +import org.openrdf.query.algebra.Join;
    +import org.openrdf.query.algebra.MultiProjection;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.Reduced;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
    +
    +public class SparqlToPipelineTransformVisitor extends QueryModelVisitorBase<Exception> {
    +    private MongoCollection<Document> inputCollection;
    +
    +    public SparqlToPipelineTransformVisitor(MongoCollection<Document> inputCollection) {
    +        this.inputCollection = inputCollection;
    --- End diff --
    
    Is this allowed to be null?


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159307808
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -63,6 +63,10 @@
         public static final String STATEMENT_METADATA = "statementMetadata";
         public static final String DOCUMENT_VISIBILITY = "documentVisibility";
     
    +    public static String hash(String value) {
    --- End diff --
    
    Docs.


---

[GitHub] incubator-rya pull request #254: RYA-416 Optionally invoke aggregation pipel...

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/254#discussion_r159336327
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java ---
    @@ -0,0 +1,882 @@
    +/*
    + * 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.aggregation;
    +
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.STATEMENT_METADATA;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.SUBJECT_HASH;
    +import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.function.Function;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.api.domain.StatementMetadata;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.operators.query.ConditionalOperators;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.Literal;
    +import org.openrdf.model.Resource;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Compare;
    +import org.openrdf.query.algebra.ExtensionElem;
    +import org.openrdf.query.algebra.ProjectionElem;
    +import org.openrdf.query.algebra.ProjectionElemList;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.ValueConstant;
    +import org.openrdf.query.algebra.ValueExpr;
    +import org.openrdf.query.algebra.Var;
    +import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.BiMap;
    +import com.google.common.collect.HashBiMap;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.model.Aggregates;
    +import com.mongodb.client.model.BsonField;
    +import com.mongodb.client.model.Filters;
    +import com.mongodb.client.model.Projections;
    +
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Represents a portion of a query tree as MongoDB aggregation pipeline. Should
    + * be built bottom-up: start with a statement pattern implemented as a $match
    + * step, then add steps to the pipeline to handle higher levels of the query
    + * tree. Methods are provided to add certain supported query operations to the
    + * end of the internal pipeline. In some cases, specific arguments may be
    + * unsupported, in which case the pipeline is unchanged and the method returns
    + * false.
    + */
    +public class AggregationPipelineQueryNode extends ExternalSet {
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to variable values.
    +     */
    +    static final String VALUES = "<VALUES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to the corresponding hashes
    +     * of their values.
    +     */
    +    static final String HASHES = "<HASHES>";
    +
    +    /**
    +     * An aggregation result corresponding to a solution should map this key
    +     * to an object which itself maps variable names to their datatypes, if any.
    +     */
    +    static final String TYPES = "<TYPES>";
    +
    +    private static final String LEVEL = "derivation_level";
    +    private static final String[] FIELDS = { VALUES, HASHES, TYPES, LEVEL, TIMESTAMP };
    +
    +    private static final String JOINED_TRIPLE = "<JOINED_TRIPLE>";
    +    private static final String FIELDS_MATCH = "<JOIN_FIELDS_MATCH>";
    +
    +    private static final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
    +
    +    private static final Bson DEFAULT_TYPE = new Document("$literal", XMLSchema.ANYURI.stringValue());
    +    private static final Bson DEFAULT_CONTEXT = new Document("$literal", "");
    +    private static final Bson DEFAULT_DV = DocumentVisibilityAdapter.toDBObject(MongoDbRdfConstants.EMPTY_DV);
    +    private static final Bson DEFAULT_METADATA = new Document("$literal",
    +            StatementMetadata.EMPTY_METADATA.toString());
    +
    +    private static boolean isValidFieldName(String name) {
    +        return !(name == null || name.contains(".") || name.contains("$")
    +                || name.equals("_id"));
    +    }
    +
    +    /**
    +     * For a given statement pattern, represents a mapping from query variables
    +     * to their corresponding parts of matching triples. If necessary, also
    +     * substitute variable names including invalid characters with temporary
    +     * replacements, while producing a map back to the original names.
    +     */
    +    private static class StatementVarMapping {
    +        private final Map<String, String> varToTripleValue = new HashMap<>();
    +        private final Map<String, String> varToTripleHash = new HashMap<>();
    +        private final Map<String, String> varToTripleType = new HashMap<>();
    +        private final BiMap<String, String> varToOriginalName;
    +
    +        String valueField(String varName) {
    +            return varToTripleValue.get(varName);
    +        }
    +        String hashField(String varName) {
    +            return varToTripleHash.get(varName);
    +        }
    +        String typeField(String varName) {
    +            return varToTripleType.get(varName);
    +        }
    +
    +        Set<String> varNames() {
    +            return varToTripleValue.keySet();
    +        }
    +
    +        private String replace(String original) {
    +            if (varToOriginalName.containsValue(original)) {
    +                return varToOriginalName.inverse().get(original);
    +            }
    +            else {
    +                String replacement = "field-" + UUID.randomUUID();
    +                varToOriginalName.put(replacement, original);
    +                return replacement;
    +            }
    +        }
    +
    +        private String sanitize(String name) {
    +            if (varToOriginalName.containsValue(name)) {
    +                return varToOriginalName.inverse().get(name);
    +            }
    +            else if (name != null && !isValidFieldName(name)) {
    +                return replace(name);
    +            }
    +            return name;
    +        }
    +
    +        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
    +            this.varToOriginalName = varToOriginalName;
    +            if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
    +                String name = sanitize(sp.getSubjectVar().getName());
    +                varToTripleValue.put(name, SUBJECT);
    +                varToTripleHash.put(name, SUBJECT_HASH);
    +            }
    +            if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
    +                String name = sanitize(sp.getPredicateVar().getName());
    +                varToTripleValue.put(name, PREDICATE);
    +                varToTripleHash.put(name, PREDICATE_HASH);
    +            }
    +            if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
    +                String name = sanitize(sp.getObjectVar().getName());
    +                varToTripleValue.put(name, OBJECT);
    +                varToTripleHash.put(name, OBJECT_HASH);
    +                varToTripleType.put(name, OBJECT_TYPE);
    +            }
    +            if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
    +                String name = sanitize(sp.getContextVar().getName());
    +                varToTripleValue.put(name, CONTEXT);
    +            }
    +        }
    +
    +        Bson getProjectExpression() {
    +            return getProjectExpression(new LinkedList<>(), str -> "$" + str);
    +        }
    +
    +        Bson getProjectExpression(Iterable<String> alsoInclude,
    +                Function<String, String> getFieldExpr) {
    +            Document values = new Document();
    +            Document hashes = new Document();
    +            Document types = new Document();
    +            for (String varName : varNames()) {
    +                values.append(varName, getFieldExpr.apply(valueField(varName)));
    +                if (varToTripleHash.containsKey(varName)) {
    +                    hashes.append(varName, getFieldExpr.apply(hashField(varName)));
    +                }
    +                if (varToTripleType.containsKey(varName)) {
    +                    types.append(varName, getFieldExpr.apply(typeField(varName)));
    +                }
    +            }
    +            for (String varName : alsoInclude) {
    +                values.append(varName, 1);
    +                hashes.append(varName, 1);
    +                types.append(varName, 1);
    +            }
    +            List<Bson> fields = new LinkedList<>();
    +            fields.add(Projections.excludeId());
    +            fields.add(Projections.computed(VALUES, values));
    +            fields.add(Projections.computed(HASHES, hashes));
    +            if (!types.isEmpty()) {
    +                fields.add(Projections.computed(TYPES, types));
    +            }
    +            fields.add(Projections.computed(LEVEL,
    +                    maxValueExpr("$" + LEVEL, getFieldExpr.apply(LEVEL), 0)));
    +            fields.add(Projections.computed(TIMESTAMP,
    +                    maxValueExpr("$" + TIMESTAMP, getFieldExpr.apply(TIMESTAMP), 0)));
    +            return Projections.fields(fields);
    +        }
    +    }
    +
    +    /**
    +     * Generate a projection expression that evaluates to the maximum of two
    +     * fields and a default value.
    +     */
    +    private static Document maxValueExpr(String field1, String field2, Object defaultValue) {
    +        if (field1.equals(field2)) {
    +            return ConditionalOperators.ifNull(field1, defaultValue);
    +        }
    +        else {
    +            Document vars = new Document("x", ConditionalOperators.ifNull(field1, defaultValue))
    +                    .append("y", ConditionalOperators.ifNull(field2, defaultValue));
    +            Document gt = new Document("$gt", Arrays.asList("$$x", "$$y"));
    +            Document maxExpr = new Document("$cond",
    +                    new Document("if", gt).append("then", "$$x").append("else", "$$y"));
    +            return new Document("$let", new Document("vars", vars).append("in", maxExpr));
    +        }
    +    }
    +
    +    /**
    +     * Given a StatementPattern, generate an object representing the arguments
    +     * to a "$match" command that will find matching triples.
    +     * @param sp The StatementPattern to search for
    +     * @param path If given, specify the field that should be matched against
    +     *  the statement pattern, using an ordered list of field names for a nested
    +     *  field. E.g. to match records { "x": { "y": <statement pattern } }, pass
    +     *  "x" followed by "y".
    +     * @return The argument of a "$match" query
    +     */
    +    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
    +        final Var subjVar = sp.getSubjectVar();
    +        final Var predVar = sp.getPredicateVar();
    +        final Var objVar = sp.getObjectVar();
    +        final Var contextVar = sp.getContextVar();
    +        RyaURI s = null;
    +        RyaURI p = null;
    +        RyaType o = null;
    +        RyaURI c = null;
    +        if (subjVar != null && subjVar.getValue() instanceof Resource) {
    +            s = RdfToRyaConversions.convertResource((Resource) subjVar.getValue());
    +        }
    +        if (predVar != null && predVar.getValue() instanceof URI) {
    +            p = RdfToRyaConversions.convertURI((URI) predVar.getValue());
    +        }
    +        if (objVar != null && objVar.getValue() != null) {
    +            o = RdfToRyaConversions.convertValue(objVar.getValue());
    +        }
    +        if (contextVar != null && contextVar.getValue() instanceof URI) {
    +            c = RdfToRyaConversions.convertURI((URI) contextVar.getValue());
    +        }
    +        RyaStatement rs = new RyaStatement(s, p, o, c);
    +        DBObject obj = strategy.getQuery(rs);
    +        // Add path prefix, if given
    +        if (path.length > 0) {
    +            StringBuilder sb = new StringBuilder();
    +            for (String str : path) {
    +                sb.append(str).append(".");
    +            }
    +            String prefix = sb.toString();
    +            Set<String> originalKeys = new HashSet<>(obj.keySet());
    +            originalKeys.forEach(key -> {
    +                Object value = obj.removeField(key);
    +                obj.put(prefix + key, value);
    +            });
    +        }
    +        return (BasicDBObject) obj;
    +    }
    +
    +    private static String valueFieldExpr(String varName) {
    +        return "$" + VALUES + "." + varName;
    +    }
    +    private static String hashFieldExpr(String varName) {
    +        return "$" + HASHES + "." + varName;
    +    }
    +    private static String typeFieldExpr(String varName) {
    +        return "$" + TYPES + "." + varName;
    +    }
    +    private static String joinFieldExpr(String triplePart) {
    +        return "$" + JOINED_TRIPLE + "." + triplePart;
    +    }
    +
    +    /**
    +     * Get an object representing the value field of some value expression, or
    +     * return null if the expression isn't supported.
    +     */
    +    private Object valueFieldExpr(ValueExpr expr) {
    +        if (expr instanceof Var) {
    +            return valueFieldExpr(((Var) expr).getName());
    +        }
    +        else if (expr instanceof ValueConstant) {
    +            return new Document("$literal", ((ValueConstant) expr).getValue().stringValue());
    +        }
    +        else {
    +            return null;
    +        }
    +    }
    +
    +    private final List<Bson> pipeline;
    +    private final MongoCollection<Document> collection;
    +    private final Set<String> assuredBindingNames;
    +    private final Set<String> bindingNames;
    +    private final BiMap<String, String> varToOriginalName;
    +
    +    private String replace(String original) {
    +        if (varToOriginalName.containsValue(original)) {
    +            return varToOriginalName.inverse().get(original);
    +        }
    +        else {
    +            String replacement = "field-" + UUID.randomUUID();
    +            varToOriginalName.put(replacement, original);
    +            return replacement;
    +        }
    +    }
    +
    +    /**
    +     * Create a pipeline based on a StatementPattern.
    +     * @param baseSP The leaf node in the query tree.
    +     */
    +    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(baseSP);
    +        this.collection = collection;
    +        this.varToOriginalName = HashBiMap.create();
    +        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
    +        this.assuredBindingNames = new HashSet<>(mapping.varNames());
    +        this.bindingNames = new HashSet<>(mapping.varNames());
    +        this.pipeline = new LinkedList<>();
    +        this.pipeline.add(Aggregates.match(getMatchExpression(baseSP)));
    +        this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
    +    }
    +
    +    AggregationPipelineQueryNode(MongoCollection<Document> collection,
    +            List<Bson> pipeline, Set<String> assuredBindingNames,
    +            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
    +        Preconditions.checkNotNull(collection);
    +        Preconditions.checkNotNull(pipeline);
    +        Preconditions.checkNotNull(assuredBindingNames);
    +        Preconditions.checkNotNull(bindingNames);
    +        Preconditions.checkNotNull(varToOriginalName);
    +        this.collection = collection;
    +        this.pipeline = pipeline;
    +        this.assuredBindingNames = assuredBindingNames;
    +        this.bindingNames = bindingNames;
    +        this.varToOriginalName = varToOriginalName;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o instanceof AggregationPipelineQueryNode) {
    +            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
    +            if (this.collection.equals(other.collection)
    +                    && this.assuredBindingNames.equals(other.assuredBindingNames)
    +                    && this.bindingNames.equals(other.bindingNames)
    +                    && this.varToOriginalName.equals(other.varToOriginalName)
    +                    && this.pipeline.size() == other.pipeline.size()) {
    +                // Check pipeline steps for equality -- underlying types don't
    +                // have well-behaved equals methods, so check for equivalent
    +                // string representations.
    +                for (int i = 0; i < this.pipeline.size(); i++) {
    +                    Bson doc1 = this.pipeline.get(i);
    +                    Bson doc2 = other.pipeline.get(i);
    +                    if (!doc1.toString().equals(doc2.toString())) {
    +                        return false;
    +                    }
    +                }
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        int result = collection.hashCode();
    +        for (Bson step : pipeline) {
    +            result = result * 37 + step.toString().hashCode();
    +        }
    +        result = result * 37 + assuredBindingNames.hashCode();
    +        result = result * 37 + bindingNames.hashCode();
    +        result = result * 37 + varToOriginalName.hashCode();
    +        return result;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
    +            throws QueryEvaluationException {
    +        return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
    +    }
    +
    +    @Override
    +    public Set<String> getAssuredBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : assuredBindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public Set<String> getBindingNames() {
    +        Set<String> names = new HashSet<>();
    +        for (String name : bindingNames) {
    +            names.add(varToOriginalName.getOrDefault(name, name));
    +        }
    +        return names;
    +    }
    +
    +    @Override
    +    public AggregationPipelineQueryNode clone() {
    +        return new AggregationPipelineQueryNode(collection,
    +                new LinkedList<>(pipeline),
    +                new HashSet<>(assuredBindingNames),
    +                new HashSet<>(bindingNames),
    +                HashBiMap.create(varToOriginalName));
    +    }
    +
    +    @Override
    +    public String getSignature() {
    +        super.getSignature();
    +        Set<String> assured = getAssuredBindingNames();
    +        Set<String> any = getBindingNames();
    +        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
    +        sb.append(String.join(", ", assured));
    +        if (any.size() > assured.size()) {
    +            Set<String> optionalBindingNames = any;
    +            optionalBindingNames.removeAll(assured);
    +            sb.append(" [")
    +                .append(String.join(", ", optionalBindingNames))
    +                .append("]");
    +        }
    +        sb.append(")\n");
    +        for (Bson doc : pipeline) {
    +            sb.append(doc.toString()).append("\n");
    +        }
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Get the internal list of aggregation pipeline steps. Note that documents
    +     * resulting from this pipeline will be structured using an internal
    +     * intermediate representation. For documents representing triples, see
    +     * {@link #getTriplePipeline}, and for query solutions, see
    +     * {@link #evaluate}.
    +     * @return The current internal pipeline.
    +     */
    +    List<Bson> getPipeline() {
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Add a join with an individual {@link StatementPattern} to the pipeline.
    +     * @param sp The statement pattern to join with
    +     * @return true if the join was successfully added to the pipeline.
    +     */
    +    public boolean joinWith(StatementPattern sp) {
    +        // 1. Determine shared variables and new variables
    +        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
    +        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
    +        sharedVars.retainAll(assuredBindingNames);
    +        // 2. Join on one shared variable
    +        String joinKey =  sharedVars.pollFirst();
    +        String collectionName = collection.getNamespace().getCollectionName();
    +        Bson join;
    +        if (joinKey == null) {
    +            return false;
    +        }
    +        else {
    +            join = Aggregates.lookup(collectionName,
    +                    HASHES + "." + joinKey,
    +                    spMap.hashField(joinKey),
    +                    JOINED_TRIPLE);
    +        }
    +        pipeline.add(join);
    +        // 3. Unwind the joined triples so each document represents a binding
    +        //   set (solution) from the base branch and a triple that may match.
    +        pipeline.add(Aggregates.unwind("$" + JOINED_TRIPLE));
    +        // 4. (Optional) If there are any shared variables that weren't used as
    +        //   the join key, project all existing fields plus a new field that
    +        //   tests the equality of those shared variables.
    +        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
    +        if (!sharedVars.isEmpty()) {
    +            List<Bson> eqTests = new LinkedList<>();
    +            for (String varName : sharedVars) {
    +                String oldField = valueFieldExpr(varName);
    +                String newField = joinFieldExpr(spMap.valueField(varName));
    +                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
    +                eqTests.add(eqTest);
    +            }
    +            Bson eqProjectOpts = Projections.fields(
    +                    Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
    +                    Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
    +            pipeline.add(Aggregates.project(eqProjectOpts));
    +            matchOpts.put(FIELDS_MATCH, true);
    +        }
    +        // 5. Filter for solutions whose triples match the joined statement
    +        //  pattern, and, if applicable, whose additional shared variables
    +        //  match the current solution.
    +        pipeline.add(Aggregates.match(matchOpts));
    +        // 5. Project the results to include variables from the new SP (with
    --- End diff --
    
    6.


---

[GitHub] incubator-rya issue #254: RYA-416 Optionally invoke aggregation pipeline to ...

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

    https://github.com/apache/incubator-rya/pull/254
  
    Just a heads up, this is going to conflict with RYA-414 because of changes to MongoDBRdfConfiguration.java.


---