You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by isper3at <gi...@git.apache.org> on 2017/06/20 17:36:13 UTC

[GitHub] incubator-rya pull request #172: [WIP] RYA-303 Mongo PCJ Support

GitHub user isper3at opened a pull request:

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

    [WIP] RYA-303 Mongo PCJ Support

    ## Description
    >What Changed?
    Added implementation of MongoPcjStorage
    Abstracted out common functionality to prevent
    code duplication
    Pcj serialization in mongo
    
    ### Tests
    >Coverage?
    Full test coverage of newly implemented mongo indexer
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/303)
    
    ### Checklist
    - [ ] Code Review
    - [x] 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/isper3at/incubator-rya RYA-303_mongo_PCJ

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

    https://github.com/apache/incubator-rya/pull/172.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 #172
    
----
commit 772ce96d774f29dacf291963a8de9cc915d22442
Author: isper3at <sm...@gmail.com>
Date:   2017-05-12T23:40:44Z

    RYA-303 Mongo PCJ Support
    
    Added implementation of MongoPcjStorage
    Abstracted out common functionality to prevent
    code duplication
    Pcj serialization in mongo
    
    full test coverage

----


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139546116
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    --- End diff --
    
    renamed to MongoIndexSet.  I feel like at some point we might be able to abstract that out?  Really depends on what solution we come up with


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534291
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoInstall.java ---
    @@ -191,26 +192,22 @@ private static MongoDBRdfConfiguration makeRyaConfig(
     
             final MongoDBRdfConfiguration conf = connectionDetails.build(ryaDetails.getRyaInstanceName());
     
    -        // The Mongo implementation of Rya does not currently support PCJs.
    -        if(ryaDetails.getPCJIndexDetails().isEnabled()) {
    -            log.warn("The install configuration says to enable PCJ indexing, but Mongo RYA does not support that " +
    -                    "feature. Ignoring this configuration.");
    -        }
    -        conf.set(ConfigUtils.USE_PCJ, "false");
    +        conf.setBoolean(ConfigUtils.USE_PCJ, ryaDetails.getPCJIndexDetails().isEnabled());
     
             // Mongo does not support entity indexing.
             if(ryaDetails.getEntityCentricIndexDetails().isEnabled()) {
                 log.warn("The install configuration says to enable Entity Centric indexing, but Mongo RYA does not support " +
                         "that feature. Ignoring this configuration.");
             }
    -        conf.set(ConfigUtils.USE_ENTITY, "false");
    +        //TODO mongo now has an entity index, just needs CLI support.
    --- End diff --
    
    discussed in person


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/653/<h2>Failed Tests: <span class='status-failure'>1</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.shell' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/653/org.apache.rya$rya.shell/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.shell</a>: <span class='status-failure'>1</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/653/org.apache.rya$rya.shell/testReport/org.apache.rya.shell.util/RyaDetailsFormatterTest/format_mongo/'><strong>org.apache.rya.shell.util.RyaDetailsFormatterTest.format_mongo</strong></a></li></ul>



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159802000
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,51 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    +    private static final Logger log = Logger.getLogger(MongoPrecomputedJoinIndexer.class);
    +
    +    @Override
    +    public MongoPcjDocuments getPcjStorage(final Configuration conf) {
    --- End diff --
    
    Why does this return an instance of MongoPcjDocuments but the method is called getPcjStorage?  Shouldn't it return an instance of PcjStorage?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132173821
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,125 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private final MongoClient client;
    +    private final MongoDBRdfConfiguration mongoConf;
    +
    +    public MongoPcjIndexSetProvider(final Configuration conf, final MongoClient client) {
    +        super(conf);
    +        this.client = client;
    +        mongoConf = new MongoDBRdfConfiguration(conf);
    +    }
    +
    +    public MongoPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices, final MongoClient client) {
    +        super(conf, indices);
    +        this.client = client;
    +        mongoConf = new MongoDBRdfConfiguration(conf);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws Exception {
    +        requireNonNull(conf);
    +        final MongoPcjDocuments pcjTables = new MongoPcjDocuments(client, mongoConf.getMongoDBName());
    +        final String pcjPrefix = requireNonNull(conf.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX));
    +        List<String> tables = null;
    +
    +        tables = mongoConf.getPcjTables();
    +        // this maps associates pcj table name with pcj sparql query
    +        final Map<String, String> indexTables = Maps.newLinkedHashMap();
    +
    +        try(final PrecomputedJoinStorage storage = new MongoPcjStorage(client, mongoConf.getMongoInstance(), null)) {
    +            final PcjTableNameFactory pcjFactory = new PcjTableNameFactory();
    +
    +            final boolean tablesProvided = tables != null && !tables.isEmpty();
    +
    +            if (tablesProvided) {
    +                // if tables provided, associate table name with sparql
    +                for (final String table : tables) {
    +                    indexTables.put(table, storage.getPcjMetadata(pcjFactory.getPcjId(table)).getSparql());
    +                }
    +            } else if (hasRyaDetails(mongoConf.getMongoDBName())) {
    +                // If this is a newer install of Rya, and it has PCJ Details,
    +                // then
    +                // use those.
    +                final List<String> ids = storage.listPcjs();
    +                for (final String id : ids) {
    +                    indexTables.put(pcjFactory.makeTableName(pcjPrefix, id), storage.getPcjMetadata(id).getSparql());
    +                }
    +            } else {
    +                // Otherwise figure it out by getting document IDs.
    +                tables = pcjTables.listPcjDocuments();
    +                for (final String table : tables) {
    +                    if (table.startsWith(pcjPrefix + "INDEX")) {
    +                        indexTables.put(table, pcjTables.getPcjMetadata(table).getSparql());
    +                    }
    +                }
    +            }
    +        }
    +
    +        // use table name sparql map (indexTables) to create {@link
    +        // AccumuloIndexSet}
    --- End diff --
    
    Why are you referencing AccumuloIndexSet here?  Is this a C&P job :)


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160496357
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,172 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private static final Logger log = Logger.getLogger(PcjQueryNode.class);
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     * Creates a new {@link PcjQueryNode}.
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     *
    +     * @throws MalformedQueryException - The SPARQL query needs to contain a projection.
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs) throws MalformedQueryException {
    --- End diff --
    
    updated docs


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132189129
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    +     * @param accCon
    +     *            - connection to a valid Accumulo instance
    +     * @param tablename
    +     *            - name of an existing PCJ table
    +     * @throws PCJStorageException
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename)
    +            throws PCJStorageException, MalformedQueryException {
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        pcjDocs = indexer.getPcjStorage(conf);
    --- End diff --
    
    Preconditions


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160517663
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    --- End diff --
    
    or [a] user specified
      


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159925407
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    +                        result = (VisibilityBindingSet) bindingSet;
    +                    } else {
    +                        //TODO warn that visibilities are being lost.  do we want to exit?
    +                        log.warn("some message");
    --- End diff --
    
    left the log and documented it


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160512621
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    --- End diff --
    
    Static import.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532262
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/AbstractPcjStorage.java ---
    @@ -0,0 +1,81 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +
    +public abstract class AbstractPcjStorage implements PrecomputedJoinStorage {
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132198496
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    --- End diff --
    
    binding_var2


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

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/243/<h2>Build result: FAILURE</span></h2>[...truncated 32.39 KB...][INFO] Apache Rya Accumulo Pig ............................ SKIPPED[INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 02:00 min[INFO] Finished at: 2017-06-22T03:51:58+00:00[INFO] Final Memory: 46M/637M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.api: Too many files with unapproved license: 1 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator
 -rya-master-with-optionals-pull-requests/common/rya.api/target/rat.txt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.apichannel stoppedSetting status of 2703c842005026f7374875538b96a5a3201b657f to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/243/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160452384
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java ---
    @@ -51,17 +51,29 @@ public static void addRyaDetailsToConfiguration(final RyaDetails details, final
     
             checkAndSet(conf, ConfigurationFields.USE_ENTITY, details.getEntityCentricIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_FREETEXT, details.getFreeTextIndexDetails().isEnabled());
    -      //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
    +        //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_TEMPORAL, details.getTemporalIndexDetails().isEnabled());
             final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -        if (pcjDetails.isEnabled() && pcjDetails.getFluoDetails().isPresent()) {
    -            checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, true);
    -            conf.set(ConfigurationFields.FLUO_APP_NAME, pcjDetails.getFluoDetails().get().getUpdateAppName());
    -            conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "FLUO");
    -            conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "ACCUMULO");
    +        
    +        if(conf.getBoolean(ConfigurationFields.USE_MONGO, false) ) {
    +        	if(pcjDetails.isEnabled()) {
    +        		conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "MONGO");
    +        		//mongo does not currently support pcj updaters
    +        		checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, false);
    +        		conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "NO_UPDATE");
    --- End diff --
    
    Same as above.  Is PrecomputedJoinUpdaterType.NO_UPDATE available to be used?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159924714
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160575734
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    --- End diff --
    
    What is a PCJ name? Did you introduce that idea to PCJ within this review?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/581/<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/581/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/581/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/581/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/581/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 #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160575656
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    --- End diff --
    
    I thought the square brackets were to to indicate a value needs to be filled in. The type uri wouldn't be a list or set.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160569265
  
    --- Diff: extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java ---
    @@ -41,7 +41,7 @@
      */
     public class MongoInstallIT extends MongoITBase {
     
    -    @Test
    +	@Test
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160514743
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoInstall.java ---
    @@ -147,22 +147,23 @@ private RyaDetails initializeRyaDetails(
                 final InstallConfiguration installConfig) throws AlreadyInitializedException, RyaDetailsRepositoryException {
             final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(adminClient, instanceName);
     
    -        // Build the PCJ Index details. [not supported in mongo]
    -        final PCJIndexDetails.Builder pcjDetailsBuilder = PCJIndexDetails.builder().setEnabled(false);
    +        if(installConfig.getFluoPcjAppName().isPresent()) {
    +        	log.warn("Mongo does not have fluo support, use ignoring the configured fluo application name: " + installConfig.getFluoPcjAppName().get());
    +        }
    +        
    +        // Build the PCJ Index details.
    +        final PCJIndexDetails.Builder pcjDetailsBuilder = PCJIndexDetails.builder()
    +                .setEnabled(installConfig.isPcjIndexEnabled());
     
             final RyaDetails details = RyaDetails.builder()
                     // General Metadata
                     .setRyaInstanceName(instanceName).setRyaVersion(getVersion())
     
    -                // FIXME RYA-215 .setGeoIndexDetails(new GeoIndexDetails(installConfig.isGeoIndexEnabled()))
    -
                     // Secondary Index Values
    +                // FIXME RYA-215 .setGeoIndexDetails(new GeoIndexDetails(installConfig.isGeoIndexEnabled()))
                     .setTemporalIndexDetails(new TemporalIndexDetails(installConfig.isTemporalIndexEnabled()))
                     .setFreeTextDetails(new FreeTextIndexDetails(installConfig.isFreeTextIndexEnabled()))//
    -
    -                // Entity centric indexing is not supported in Mongo DB.
    -                .setEntityCentricIndexDetails(new EntityCentricIndexDetails(false))
    -
    +                .setEntityCentricIndexDetails(new EntityCentricIndexDetails(installConfig.isEntityCentrixIndexEnabled()))
    --- End diff --
    
    Did you enable entity centric indexing in this ticket too? Shouldn't that be a separate ticket?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761412
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160547307
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    --- End diff --
    
    preconditions


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160569137
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            final MongoClient client = mongoConf.getMongoClient();
    +            final MongoPcjDocuments pcjDocs = new MongoPcjDocuments(client, mongoConf.getRyaInstanceName());
    +            List<String> tables = null;
    --- End diff --
    
    there should really only be the PCJ collection.  I may take a look at just cleaning this up to use the PCJ collection


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160514908
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoInstall.java ---
    @@ -191,26 +192,22 @@ private static MongoDBRdfConfiguration makeRyaConfig(
     
             final MongoDBRdfConfiguration conf = connectionDetails.build(ryaDetails.getRyaInstanceName());
     
    -        // The Mongo implementation of Rya does not currently support PCJs.
    -        if(ryaDetails.getPCJIndexDetails().isEnabled()) {
    -            log.warn("The install configuration says to enable PCJ indexing, but Mongo RYA does not support that " +
    -                    "feature. Ignoring this configuration.");
    -        }
    -        conf.set(ConfigUtils.USE_PCJ, "false");
    +        conf.setBoolean(ConfigUtils.USE_PCJ, ryaDetails.getPCJIndexDetails().isEnabled());
     
             // Mongo does not support entity indexing.
             if(ryaDetails.getEntityCentricIndexDetails().isEnabled()) {
                 log.warn("The install configuration says to enable Entity Centric indexing, but Mongo RYA does not support " +
                         "that feature. Ignoring this configuration.");
             }
    -        conf.set(ConfigUtils.USE_ENTITY, "false");
    +        //TODO mongo now has an entity index, just needs CLI support.
    --- End diff --
    
    Address the TODOs and out of date comments in there. Should figure out if we're doing entity centric stuff in this ticket or not.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831044
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    --- End diff --
    
    PCJ_ID


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159936556
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjIndexer.java ---
    @@ -0,0 +1,38 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +
    +import edu.umd.cs.findbugs.annotations.Nullable;
    +
    +/**
    + * Updates the PCJs that are in a {@link MongoPcjDocuments}.
    + */
    +public interface PcjIndexer extends RyaSecondaryIndexer {
    +    /**
    +     * Creates the {@link MongoPcjDocuments} that will be used by the indexer.
    +     *
    +     * @param conf - Indicates how the {@link MongoPcjDocuments} is initialized. (not null)
    +     * @return The {@link MongoPcjDocuments} that will be used by this indexer.
    +     */
    +    public @Nullable MongoPcjDocuments getPcjStorage(Configuration conf);
    --- End diff --
    
    Seems like the primary purpose of this class is to parse the config object to create the MongoPcjDocuments object.  I don't think that this functionality should be lumped in with the role of the Indexer.  In the case of PCJs, the primary role of the indexer is to route statements to the updater service, which then routes Bindingset results to Kafka, Mongo, Accumulo, etc for use with PCJs.  You are primarily creating and using the indexer for the purpose of parsing the config object to create the MongoPcjDocuments object.  You are not using it for its primary purpose, which is confusing.  Also, you are adding a Mongo specific method to a very general indexing interface and not indicating that the extension is Mongo specific.  This is also very misleading.  I think all of this would be much cleaner if you separated the parsing of config/creation of MongoPcjDocuments from the indexing API.  By combining them you are abusing established patterns of usage.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160830949
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    --- End diff --
    
    pcj name


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159931217
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java ---
    @@ -1,221 +0,0 @@
    -/**
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - *     http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing,
    - * software distributed under the License is distributed on an
    - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    - * KIND, either express or implied.  See the License for the
    - * specific language governing permissions and limitations
    - * under the License.
    - */
    -package org.apache.rya.indexing.pcj.matching;
    -
    -import java.util.ArrayList;
    -import java.util.Iterator;
    -import java.util.List;
    -import java.util.Map;
    -import java.util.Objects;
    -
    -import org.apache.accumulo.core.client.AccumuloException;
    -import org.apache.accumulo.core.client.AccumuloSecurityException;
    -import org.apache.accumulo.core.client.Connector;
    -import org.apache.accumulo.core.client.TableNotFoundException;
    -import org.apache.hadoop.conf.Configuration;
    -import org.apache.log4j.Logger;
    -import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
    -import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    -import org.apache.rya.api.instance.RyaDetailsRepository;
    -import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    -import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    -import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    -import org.apache.rya.indexing.accumulo.ConfigUtils;
    -import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    -import org.apache.rya.indexing.external.matching.QuerySegment;
    -import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
    -import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    -import org.apache.rya.indexing.pcj.storage.PcjException;
    -import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    -import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
    -import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    -import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
    -import org.openrdf.query.MalformedQueryException;
    -import org.openrdf.query.QueryEvaluationException;
    -import org.openrdf.query.algebra.TupleExpr;
    -import org.openrdf.sail.SailException;
    -
    -import com.google.common.base.Preconditions;
    -import com.google.common.collect.Lists;
    -import com.google.common.collect.Maps;
    -
    -/**
    - * Implementation of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    - * This provider uses either user specified Accumulo configuration information or user a specified
    - * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If Accumulo configuration
    - * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    - * PCJs registered in RyaDetails.
    - *
    - */
    -public class AccumuloIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    --- End diff --
    
    oh, I repackaged it into a matching.provider package


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160532247
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/instance/MongoDetailsAdapter.java ---
    @@ -186,7 +180,7 @@ public static RyaDetails toRyaDetails(final DBObject mongoObj) throws MalformedR
             if (!pcjIndexDBO.getBoolean(PCJ_ENABLED_KEY)) {
                 pcjBuilder.setEnabled(false);
             } else {
    -            pcjBuilder.setEnabled(true).setFluoDetails(new FluoDetails(pcjIndexDBO.getString(PCJ_FLUO_KEY)));
    +            pcjBuilder.setEnabled(true);//no fluo details to set since mongo has no fluo support
    --- End diff --
    
    I'll change the comment, but same response


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r129946162
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    --- End diff --
    
    Is this right?  Looks like a copy and paste from the entity index.  


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570620
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    --- End diff --
    
    well its not really part of a factory, but yeah, i can make it make


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160793098
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    You're re-using the idea of PCJ ID, which is defined as the ID that uniquely identifies the PCJ within Rya (which I honestly think is your PCJ Name in here), to mean something else.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532054
  
    --- Diff: extras/rya.benchmark/src/main/gen/org/apache/rya/benchmark/query/ObjectFactory.java ---
    @@ -20,7 +20,7 @@
     // This file was generated by the JavaTM Architecture for XML Binding(JAXB) Reference Implementation, v2.2.11 
     // See <a href="http://java.sun.com/xml/jaxb">http://java.sun.com/xml/jaxb</a> 
     // Any modifications to this file will be lost upon recompilation of the source schema. 
    -// Generated on: 2016.12.16 at 01:22:14 PM PST 
    +// Generated on: 2017.07.06 at 03:13:11 PM EDT 
    --- End diff --
    
    uhhhh


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160509984
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -154,6 +154,5 @@ public StatefulMongoDBRdfConfiguration getConf() {
     
         @Override
         public void close() throws IOException {
    -//        if (mongoClient != null){ mongoClient.close(); }
    --- End diff --
    
    Delete commented out code.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160762007
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -320,11 +343,14 @@ public String installWithMongoParameters(
                 final boolean enableFreeTextIndex,
     
                 // TODO RYA-215
    -//            @CliOption(key = {"enableGeospatialIndex"}, mandatory = false, help = "Use Geospatial Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    -//            final boolean enableGeospatialIndex,
    +            //            @CliOption(key = {"enableGeospatialIndex"}, mandatory = false, help = "Use Geospatial Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    +            //            final boolean enableGeospatialIndex,
     
                 @CliOption(key = {"enableTemporalIndex"}, mandatory = false, help = "Use Temporal Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    -            final boolean enableTemporalIndex) {
    +            final boolean enableTemporalIndex,
    --- End diff --
    
    I went ahead and just reverted it, was faster to just take care of it later


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160834195
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160516315
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    --- End diff --
    
    This class still has a bunch of Entity index stuff in it, so that needs to be sorted out. It's also very MongoDB specific, so why does it exist? It can't be a general base class.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160550261
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -320,11 +343,14 @@ public String installWithMongoParameters(
                 final boolean enableFreeTextIndex,
     
                 // TODO RYA-215
    -//            @CliOption(key = {"enableGeospatialIndex"}, mandatory = false, help = "Use Geospatial Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    -//            final boolean enableGeospatialIndex,
    +            //            @CliOption(key = {"enableGeospatialIndex"}, mandatory = false, help = "Use Geospatial Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    +            //            final boolean enableGeospatialIndex,
     
                 @CliOption(key = {"enableTemporalIndex"}, mandatory = false, help = "Use Temporal Indexing.", unspecifiedDefaultValue = "false", specifiedDefaultValue = "true")
    -            final boolean enableTemporalIndex) {
    +            final boolean enableTemporalIndex,
    --- End diff --
    
    Since you added entity stuff back in, also add it here?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127529921
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     *
    +     * @return Iterator of Lists (combos) of PCJs used to build an optimal query plan
    +     */
    +    @Override
    +    public Iterator<List<ExternalTupleSet>> getExternalSetCombos(final QuerySegment<ExternalTupleSet> segment) {
    +        final ValidIndexCombinationGenerator comboGen = new ValidIndexCombinationGenerator(segment.getOrderedNodes());
    +        return comboGen.getValidIndexCombos(getExternalSets(segment));
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     * @return List of PCJs for matching
    +     */
    +    @Override
    +    public List<ExternalTupleSet> getExternalSets(final QuerySegment<ExternalTupleSet> segment) {
    +        try {
    +            if(!init) {
    +                indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +                init = true;
    +            }
    +            final TupleExpr query = segment.getQuery().getTupleExpr();
    +            final IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(query, indexCache);
    +            final List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
    +            final List<ExternalTupleSet> tuples = new ArrayList<>();
    +            for (final ExternalTupleSet tuple: pcjs) {
    +                final QuerySegment<ExternalTupleSet> pcj = converter.setToSegment(tuple);
    +                if (segment.containsQuerySegment(pcj)) {
    +                    tuples.add(tuple);
    +                }
    +            }
    +            return tuples;
    +
    +        } catch (final Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    public int size() throws Exception {
    +        if (!init) {
    +            indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +            init = true;
    +        }
    +        return indexCache.size();
    +    }
    +
    +    /**
    +     *
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532527
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    public Document getMetadataDocument(final String pcjName, final String sparql) {
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn
    --- End diff --
    
    nit: fix


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

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/251/<h2>Build result: FAILURE</span></h2>[...truncated 4.36 MB...][INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 08:18 min[INFO] Finished at: 2017-07-06T19:25:20+00:00[INFO] Final Memory: 135M/848M[INFO] ------------------------------------------------------------------------Waiting for Jenkins to finish collecting data[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.indexing.pcj: Too many files with unapproved license: 3 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-w
 ith-optionals-pull-requests/extras/rya.indexing.pcj/target/rat.txt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.indexing.pcjchannel stoppedSetting status of 93e29d4c57c487813cd0b46e94f5587d30cc4973 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/251/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532493
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    public Document getMetadataDocument(final String pcjName, final String sparql) {
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r161103080
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,446 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_ID]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_ID],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_METADATA_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_ID = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_METADATA_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ documents will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the PCJ results document. (not null)
    +     * @throws PCJStorageException The PCJ documents could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ results document.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ Id.
    +     *
    +     * @param pcjId - The Id of the PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ metadata document does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjId) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_METADATA_ID, makeMetadataID(pcjId))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjId + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param results - The binding set results. (not null)
    +     */
    +    public void addResults(final String pcjId, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjId);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_ID, pcjId);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_METADATA_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ results document with the provided name.
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160569466
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    --- End diff --
    
    freaking.....git dropping more stuff.  I need to figure out what I did and why it dropped everything


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160790192
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        try {
    +            //TODO use the PCJ collection.
    --- End diff --
    
    Is this TODO still relevant? Isn't that what RyaDetails is for? What do you actually mean here? Maybe make a jira ticket for it and explain the improvement if this isn't going to be addressed.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/648/<h2>Failed Tests: <span class='status-failure'>1</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.shell' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/648/org.apache.rya$rya.shell/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.shell</a>: <span class='status-failure'>1</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/648/org.apache.rya$rya.shell/testReport/org.apache.rya.shell.util/RyaDetailsFormatterTest/format_mongo/'><strong>org.apache.rya.shell.util.RyaDetailsFormatterTest.format_mongo</strong></a></li></ul>



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831320
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    +     * @param sparql
    +     *            - The query the pcj is assigned to.
    +     * @throws @throws
    +     *             PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    --- End diff --
    
    provided PCJ [ID].


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160494487
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java ---
    @@ -51,17 +51,29 @@ public static void addRyaDetailsToConfiguration(final RyaDetails details, final
     
             checkAndSet(conf, ConfigurationFields.USE_ENTITY, details.getEntityCentricIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_FREETEXT, details.getFreeTextIndexDetails().isEnabled());
    -      //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
    +        //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_TEMPORAL, details.getTemporalIndexDetails().isEnabled());
             final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -        if (pcjDetails.isEnabled() && pcjDetails.getFluoDetails().isPresent()) {
    -            checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, true);
    -            conf.set(ConfigurationFields.FLUO_APP_NAME, pcjDetails.getFluoDetails().get().getUpdateAppName());
    -            conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "FLUO");
    -            conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "ACCUMULO");
    +        
    +        if(conf.getBoolean(ConfigurationFields.USE_MONGO, false) ) {
    +        	if(pcjDetails.isEnabled()) {
    +        		conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "MONGO");
    +        		//mongo does not currently support pcj updaters
    +        		checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, false);
    +        		conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "NO_UPDATE");
    --- End diff --
    
    same as above


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160494776
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    --- End diff --
    
    I don't think so?  I used the same number from the accumulo implementation


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528668
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532103
  
    --- Diff: extras/rya.benchmark/src/main/gen/org/apache/rya/benchmark/query/Parameters.java ---
    @@ -20,7 +20,7 @@
     // This file was generated by the JavaTM Architecture for XML Binding(JAXB) Reference Implementation, v2.2.11 
     // See <a href="http://java.sun.com/xml/jaxb">http://java.sun.com/xml/jaxb</a> 
     // Any modifications to this file will be lost upon recompilation of the source schema. 
    -// Generated on: 2016.12.16 at 01:22:14 PM PST 
    +// Generated on: 2017.07.06 at 03:13:11 PM EDT 
    --- End diff --
    
    huh?


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160455634
  
    --- Diff: extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java ---
    @@ -78,6 +78,47 @@ public void install() throws DuplicateInstanceNameException, RyaClientException
             assertTrue("Tables missing from:" + expected + " actual:" + found, expected.size() == count);
             assertTrue("Instance should exist.", instanceExists.exists(ryaInstance));
         }
    +	
    +	@Test
    --- End diff --
    
    Spacing off


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160832192
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    +     * @param sparql
    +     *            - The query the pcj is assigned to.
    +     * @throws @throws
    +     *             PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjId - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjId) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjId))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjId + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * 
    +     * @param pcjId
    +     *            - The PCJ to add the results to.
    +     * @param results
    +     *            - The binding set results.
    +     */
    +    public void addResults(final String pcjId, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjId);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjId);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjId) {
    +        requireNonNull(pcjId);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjId);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ
    +     * document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjId
    +     *            - The name of the PCJ table that will receive the results. (not
    +     *            null)
    +     * @param ryaConn
    +     *            - A connection to the Rya store that will be queried to find
    +     *            results. (not null)
    +     * @throws PCJStorageException
    +     *             If results could not be written to the PCJ table, the PCJ table
    +     *             does not exist, or the query that is being execute was malformed.
    +     */
    +    public void populatePcj(final String pcjId, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjId);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjId);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                if(batch.size() == 1000) {
    +                    addResults(pcjId, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjId, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException(
    +                    "Could not populate a PCJ document with Rya results for the pcj named: " + pcjId, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjIds = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjIds.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjIds;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjId
    +     *            - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjId) {
    +        requireNonNull(pcjId);
    +
    +        // get all results based on pcjId
    +        return queryForBindings(new Document(PCJ_NAME, pcjId));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjId.
    +     *
    +     * @param pcjId
    +     *            - The pcj to retrieve results for.
    +     * @param authorizations
    +     *            - The authorizations of the user to restrict results.
    +     * @param restrictionBindings
    +     *            - The collection of {@link BindingSet}s to restrict results.
    +     *            <p>
    +     *            Note: the result restrictions from {@link BindingSet}s are an OR
    +     *            over ANDS in that: <code>
    +     *  [
    +     *     bindingset: binding AND binding AND binding,
    +     *     OR
    +     *     bindingset: binding AND binding AND binding,
    +     *     .
    +     *     .
    +     *     .
    +     *     OR
    +     *     bindingset: binding
    +     *  ]
    +     * </code>
    +     * @return
    +     */
    +    public CloseableIterator<BindingSet> getResults(final String pcjId, final Authorizations authorizations,
    +            final Collection<BindingSet> restrictionBindings) {
    +        // empty bindings return all results.
    +        if (restrictionBindings.size() == 1 && restrictionBindings.iterator().next().size() == 0) {
    +            return listResults(pcjId);
    +        }
    +
    +        final Document query = new Document(PCJ_NAME, pcjId);
    +        final Document bindingSetDoc = new Document();
    +        final List<Document> bindingSetList = new ArrayList<>();
    +        restrictionBindings.forEach(bindingSet -> {
    +            final Document bindingDoc = new Document();
    +            final List<Document> bindings = new ArrayList<>();
    +            bindingSet.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                final Document typeDoc = new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData());
    +                final Document bind = new Document(binding.getName(), typeDoc);
    +                bindings.add(bind);
    +            });
    +            bindingDoc.append("$and", bindings);
    +            bindingSetList.add(bindingDoc);
    +        });
    +        bindingSetDoc.append("$or", bindingSetList);
    +        return queryForBindings(query);
    +    }
    +
    +    private CloseableIterator<BindingSet> queryForBindings(final Document query) {
    +        final FindIterable<Document> rez = pcjCollection.find(query);
    +        final Iterator<Document> resultsIter = rez.iterator();
    +        return new CloseableIterator<BindingSet>() {
    +            @Override
    +            public boolean hasNext() {
    +                return resultsIter.hasNext();
    +            }
    +
    +            @Override
    +            public BindingSet next() {
    +                final Document bs = resultsIter.next();
    +                final MapBindingSet binding = new MapBindingSet();
    +                for (final String key : bs.keySet()) {
    +                    if (key.equals(VISIBILITIES_FIELD)) {
    +                        // has auths, is a visibility binding set.
    +                    } else if (!key.equals("_id") && !key.equals(PCJ_NAME)) {
    +                        // is the binding value.
    +                        final Document typeDoc = (Document) bs.get(key);
    +                        final URI dataType = new URIImpl(typeDoc.getString(BINDING_TYPE));
    +                        final RyaType type = new RyaType(dataType, typeDoc.getString(BINDING_VALUE));
    +                        final Value value = RyaToRdfConversions.convertValue(type);
    +                        binding.addBinding(key, value);
    +                    }
    +                }
    +                return binding;
    +            }
    +
    +            @Override
    +            public void close() throws Exception {
    +            }
    +        };
    +    }
    +
    +    /**
    +     * Drops a pcj based on the PCJ name. Removing the entire document from Mongo.
    --- End diff --
    
    ... based on the PCJ [ID].


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139545751
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    --- End diff --
    
    The BasePcjIndexer does all the heavy lifting, this implementation allows the base to know which repo to connect to


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570170
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    --- End diff --
    
    nvm.  didn't look right


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532674
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoVisibilityBindingSetBsonConverter.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.DOCUMENT_VISIBILITY_KEY;
    +
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.MalformedDocumentVisibilityException;
    +import org.bson.BsonArray;
    +import org.bson.BsonDocument;
    +import org.bson.BsonString;
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.impl.MapBindingSet;
    +
    +import com.mongodb.DBObject;
    +import com.mongodb.MongoClient;
    +import com.mongodb.util.JSON;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Converts {@link BindingSet}s to Strings and back again. The Strings do not
    + * include the binding names and are ordered with a {@link VariableOrder}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoVisibilityBindingSetBsonConverter {/* implements MongoBindingSetConverter {
    --- End diff --
    
    huh


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

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    It's part of the shell. You need to update it since we've changed what we support via the shell in this review.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160527700
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    --- End diff --
    
    Do the supers not check if this is null?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160510946
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/instance/MongoDetailsAdapter.java ---
    @@ -124,11 +123,6 @@ private static DBObject toDBObject(final PCJIndexDetails pcjIndexDetails) {
             // Is Enabled
             builder.add(PCJ_ENABLED_KEY, pcjIndexDetails.isEnabled());
     
    -        // Fluo Details if present.
    -        if(pcjIndexDetails.getFluoDetails().isPresent()) {
    --- End diff --
    
    I don't think we should be deleting conversion code just because we do not support this piece of information in Mongo DB currently. The class advertises itself as a converter from RyaDetails object to BasicDBObject. Either the RyaDetails used by Mongo should be different from that used by Accumulo (I don't think this is the route we should go), or some other hunk of code should ensure the fluo details are never present before it even gets to this point.
      


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160509796
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java ---
    @@ -40,8 +40,8 @@
      */
     public class RyaToRdfConversions {
     
    -    public static URI convertURI(RyaURI uri) {
    -        return new URIImpl(uri.getData());
    +    public static URI convertURI(RyaType value) {
    --- End diff --
    
    Docs


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160511290
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/instance/MongoDetailsAdapter.java ---
    @@ -186,7 +180,7 @@ public static RyaDetails toRyaDetails(final DBObject mongoObj) throws MalformedR
             if (!pcjIndexDBO.getBoolean(PCJ_ENABLED_KEY)) {
                 pcjBuilder.setEnabled(false);
             } else {
    -            pcjBuilder.setEnabled(true).setFluoDetails(new FluoDetails(pcjIndexDBO.getString(PCJ_FLUO_KEY)));
    +            pcjBuilder.setEnabled(true);//no fluo details to set since mongo has no fluo support
    --- End diff --
    
    Same comment down here.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132183937
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    --- End diff --
    
    It appears this class does nothing.  Is this just a stub class for when an observer framework is in place?


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528567
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160813316
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjStorage.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PCJIdFactory;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.BindingSet;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A mongo backed implementation of {@link PrecomputedJoinStorage}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPcjStorage implements PrecomputedJoinStorage {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +    // Used to update the instance's metadata.
    +    private final MongoRyaInstanceDetailsRepository ryaDetailsRepo;
    +
    +    private final String ryaInstanceName;
    +
    +    // Factories that are used to create new PCJs.
    +    private final PCJIdFactory pcjIdFactory = new PCJIdFactory();
    +
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     * Constructs an instance of {@link MongoPcjStorage}.
    +     *
    +     * @param client - The {@link MongoClient} that will be used to connect to Mongodb. (not null)
    +     * @param ryaInstanceName - The name of the RYA instance that will be accessed. (not null)
    +     */
    +    public MongoPcjStorage(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        this.ryaInstanceName = requireNonNull(ryaInstanceName);
    +        pcjDocs = new MongoPcjDocuments(client, ryaInstanceName);
    +        ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +    }
    +
    +    @Override
    +    public String createPcj(final String sparql) throws PCJStorageException {
    +        requireNonNull(sparql);
    +
    +        // Update the Rya Details for this instance to include the new PCJ
    +        // table.
    +        final String pcjId = pcjIdFactory.nextId();
    +
    +        try {
    +            new RyaDetailsUpdater(ryaDetailsRepo).update(originalDetails -> {
    +                // Create the new PCJ's details.
    +                final PCJDetails.Builder newPcjDetails = PCJDetails.builder().setId(pcjId);
    +
    +                // Add them to the instance's details.
    +                final RyaDetails.Builder mutated = RyaDetails.builder(originalDetails);
    +                mutated.getPCJIndexDetails().addPCJDetails(newPcjDetails);
    +                return mutated.build();
    +            });
    +        } catch (final RyaDetailsRepositoryException | CouldNotApplyMutationException e) {
    +            throw new PCJStorageException(String.format("Could not create a new PCJ for Rya instance '%s' "
    +                    + "because of a problem while updating the instance's details.", ryaInstanceName), e);
    +        }
    +
    +        // Create the objectID of the document to house the PCJ results.
    +        pcjDocs.createPcj(pcjId, sparql);
    --- End diff --
    
    I'll change it to ID everywehre


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160834091
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132190366
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    +     * @param accCon
    +     *            - connection to a valid Accumulo instance
    +     * @param tablename
    +     *            - name of an existing PCJ table
    +     * @throws PCJStorageException
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename)
    +            throws PCJStorageException, MalformedQueryException {
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        pcjDocs = indexer.getPcjStorage(conf);
    +        this.tablename = tablename;
    +    }
    +
    +    /**
    +     * returns size of table for query planning
    +     */
    +    @Override
    +    public double cardinality() {
    +        double cardinality = 0;
    +        try {
    +            cardinality = pcjDocs.getPcjMetadata(tablename).getCardinality();
    +        } catch (final PcjException e) {
    +            e.printStackTrace();
    +        }
    +        return cardinality;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindingset)
    +            throws QueryEvaluationException {
    +        return this.evaluate(Collections.singleton(bindingset));
    +    }
    +
    +    /**
    +     * Core evaluation method used during query evaluation - given a collection
    +     * of binding set constraints, this method finds common binding labels
    +     * between the constraints and table, uses those to build a prefix scan of
    +     * the Accumulo table, and creates a solution binding set by iterating of
    +     * the scan results.
    +     *
    +     * @param bindingset
    +     *            - collection of {@link BindingSet}s to be joined with PCJ
    +     * @return - CloseableIteration over joined results
    +     */
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final Collection<BindingSet> bindingset)
    +            throws QueryEvaluationException {
    +
    +        if (bindingset.isEmpty()) {
    +            return new IteratorWrapper<BindingSet, QueryEvaluationException>(new HashSet<BindingSet>().iterator());
    +        }
    +        final CloseableIterator<BindingSet> iter = pcjDocs.getResults(tablename, new Authorizations(), bindingset);
    --- End diff --
    
    You need to be able to handle Authorizations here.  Maybe this is okay for a first pass. We'll have to fix this when we have an updater to create BindingSet visibilities.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160546626
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    --- End diff --
    
    pcjCollectionName or pcjId?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160760303
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private final MongoConnectionDetails connectionDetails;
    --- End diff --
    
    removed


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534095
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,96 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoDeletePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoDeletePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoDeletePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public void deletePCJ(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    +        final MongoCredential cred;
    +        final MongoClient client;
    +
    +        final ServerAddress addr = new ServerAddress(connectionDetails.getHostname(), connectionDetails.getPort());
    +        if(username.isPresent()) {
    +            cred = MongoCredential.createCredential(username.get(), ryaInstanceName, connectionDetails.getPassword().get());
    +            client = new MongoClient(addr, Lists.newArrayList(cred));
    +        } else {
    +            client = new MongoClient(addr);
    +        }
    +
    +
    +        //pcjStore doesn't need to be closed since it doesn't need to release any resources.
    +        @SuppressWarnings("resource")
    +        final MongoPcjStorage pcjStore = new MongoPcjStorage(client, ryaInstanceName);
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160755378
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    --- End diff --
    
    pcjId


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160544633
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -104,11 +104,11 @@ public RyaAdminCommands(
             INSTALL_CMD})
         public boolean areStorageCommandsAvailable() {
             switch(state.getShellState().getConnectionState()) {
    -            case CONNECTED_TO_STORAGE:
    -            case CONNECTED_TO_INSTANCE:
    -                return true;
    -            default:
    -                return false;
    +        case CONNECTED_TO_STORAGE:
    --- End diff --
    
    white space changed


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160514459
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,96 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoDeletePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoDeletePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoDeletePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public void deletePCJ(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    --- End diff --
    
    Same comment about not creating a client in here.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570226
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160196480
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -141,7 +174,8 @@ public boolean arePCJCommandsAvailable() {
             // The PCJ commands are only available if the Shell is connected to an instance of Rya
             // that is new enough to use the RyaDetailsRepository and is configured to maintain PCJs.
             final ShellState shellState = state.getShellState();
    -        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE) {
    +        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE &&
    +                shellState.getStorageType().get() == StorageType.ACCUMULO) {
    --- End diff --
    
    I'm not too concerned with a lot of those since this PR is the last in the expected merge order.  If it causes any problems, it'll show up with a merge conflict and I'll take care of it then


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/640/<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/640/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/640/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/640/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/640/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 issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761918
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -104,11 +104,11 @@ public RyaAdminCommands(
             INSTALL_CMD})
         public boolean areStorageCommandsAvailable() {
             switch(state.getShellState().getConnectionState()) {
    -            case CONNECTED_TO_STORAGE:
    -            case CONNECTED_TO_INSTANCE:
    -                return true;
    -            default:
    -                return false;
    +        case CONNECTED_TO_STORAGE:
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532967
  
    --- Diff: extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocumentsTest.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.mongodb.MockMongoFactory;
    +import org.bson.Document;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.impl.MapBindingSet;
    +
    +import com.mongodb.MongoClient;
    +
    +public class MongoPcjDocumentsTest {
    +    private static final String PCJ_TEST = "pcjTest";
    +
    +    private MongoClient client;
    +    private MongoPcjDocuments docConverter;
    +
    +    @Before
    +    public void setup() throws Exception {
    +        client = MockMongoFactory.newFactory().newMongoClient();
    +        docConverter = new MongoPcjDocuments(client, PCJ_TEST);
    +    }
    +
    +    @After
    +    public void cleanup() throws Exception {
    +        client.close();
    +    }
    +
    +    @Test
    +    public void pcjToMetadata() {
    +        final String sparql = "SELECT * WHERE { ?a <http://isA> ?b }";
    +        final Document actual = docConverter.getMetadataDocument("pcjTest", sparql);
    +        final Document expected = new Document()
    +                .append(MongoPcjDocuments.CARDINALITY_FIELD, 0)
    +                .append(MongoPcjDocuments.PCJ_ID, "pcjTest_METADATA")
    +                .append(MongoPcjDocuments.SPARQL_FIELD, sparql);
    +        assertEquals(expected, actual);
    +    }
    +
    +    @Test
    +    public void metadataExists() throws Exception {
    +        final String sparql = "SELECT * WHERE { ?a <http://isA> ?b }";
    +        docConverter.createPcj("pcjTest", sparql);
    +
    +        PcjMetadata actual = docConverter.getPcjMetadata("pcjTest");
    +        PcjMetadata expected = new PcjMetadata(sparql, 0, new ArrayList<VariableOrder>());
    +        assertEquals(expected, actual);
    +
    +        // Setup the binding set that will be converted.
    +        final MapBindingSet originalBindingSet1 = new MapBindingSet();
    +        originalBindingSet1.addBinding("x", new URIImpl("http://a"));
    +        originalBindingSet1.addBinding("y", new URIImpl("http://b"));
    +        originalBindingSet1.addBinding("z", new URIImpl("http://c"));
    +        final VisibilityBindingSet results1 = new VisibilityBindingSet(originalBindingSet1, "A&B&C");
    +
    +        // Setup the binding set that will be converted.
    +        final MapBindingSet originalBindingSet2 = new MapBindingSet();
    +        originalBindingSet2.addBinding("x", new URIImpl("http://1"));
    +        originalBindingSet2.addBinding("y", new URIImpl("http://2"));
    +        originalBindingSet2.addBinding("z", new URIImpl("http://3"));
    +        final VisibilityBindingSet results2 = new VisibilityBindingSet(originalBindingSet2, "A&B&C");
    +
    +        final List<VisibilityBindingSet> bindingSets = new ArrayList<>();
    +        bindingSets.add(results1);
    +        bindingSets.add(results2);
    +
    +        docConverter.addResults("pcjTest", bindingSets);
    +        actual = docConverter.getPcjMetadata("pcjTest");
    +        expected = new PcjMetadata(sparql, 2, new ArrayList<VariableOrder>());
    +        assertEquals(expected, actual);
    +
    +        docConverter.purgePcjs("pcjTest");
    +        actual = docConverter.getPcjMetadata("pcjTest");
    +        expected = new PcjMetadata(sparql, 0, new ArrayList<VariableOrder>());
    +        assertEquals(expected, actual);
    +    }
    +
    +    /*
    +     * @Test public void bindingsToBson_Test() throws
    --- End diff --
    
    remove


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139547099
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjAdapter.java ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.BsonArray;
    +import org.bson.BsonDocument;
    +import org.bson.BsonString;
    +import org.bson.BsonValue;
    +import org.bson.Document;
    +import org.bson.codecs.DocumentCodec;
    +import org.bson.codecs.configuration.CodecRegistries;
    +import org.bson.codecs.configuration.CodecRegistry;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +/**
    + * Converts a Pcj for storage in mongoDB or retrieval from mongoDB.
    + */
    +public class MongoPcjAdapter implements BindingSetConverter<Bson> {
    --- End diff --
    
    deleted.  its not used.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160537019
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    --- End diff --
    
    uri is kind of a weird name for this. rdfType or something would be better


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160829286
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/RyaDetailsFormatter.java ---
    @@ -72,30 +72,28 @@ public String format(StorageType storageType, final RyaDetails details) {
             report.append("  Temporal Index:\n");
             report.append("    Enabled: ").append( details.getTemporalIndexDetails().isEnabled() ).append("\n");
     
    -        if(storageType == StorageType.ACCUMULO) {
    -            report.append("  PCJ Index:\n");
    -            final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -            report.append("    Enabled: ").append( pcjDetails.isEnabled() ).append("\n");
    -            if(pcjDetails.isEnabled()) {
    -                if(pcjDetails.getFluoDetails().isPresent()) {
    -                    final String fluoAppName = pcjDetails.getFluoDetails().get().getUpdateAppName();
    -                    report.append("    Fluo App Name: ").append(fluoAppName).append("\n");
    -                }
    +        report.append("  PCJ Index:\n");
    --- End diff --
    
    Yea, everything under it.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160537377
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    --- End diff --
    
    Is this for the pcj id? Could we call it pcjId instead of pcjName?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831146
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    --- End diff --
    
    Update docs. Uniquely identifies a PCJ within Rya.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160762129
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/InstallPrompt.java ---
    @@ -91,14 +91,14 @@ public InstallConfiguration promptInstallConfiguration(final String instanceName
                 checkState(storageType.isPresent(), "The shell must be connected to a storage to use the install prompt.");
     
                 switch(sharedShellState.getShellState().getStorageType().get()) {
    -                case ACCUMULO:
    -                    return promptAccumuloConfig(instanceName);
    +            case ACCUMULO:
    +                return promptAccumuloConfig(instanceName);
     
    -                case MONGO:
    -                    return promptMongoConfig(instanceName);
    +            case MONGO:
    +                return promptMongoConfig(instanceName);
     
    -                default:
    -                    throw new IllegalStateException("Unsupported storage type: " + storageType.get());
    +            default:
    +                throw new IllegalStateException("Unsupported storage type: " + storageType.get());
    --- End diff --
    
    done


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160453784
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    +
    +        final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
    +            .collect(groupingBy(RyaStatement::getSubject));
    +
    +        for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
    +            try {
    +                updateEntity(entry.getKey(), entry.getValue());
    +            } catch (final EntityStorageException e) {
    +                throw new IOException("Failed to update the Entity index.", e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Updates a {@link Entity} to reflect new {@link RyaStatement}s.
    +     *
    +     * @param subject - The Subject of the {@link Entity} the statements are for. (not null)
    +     * @param statements - Statements that the {@link Entity} will be updated with. (not null)
    +     */
    +    private void updateEntity(final RyaURI subject, final Collection<RyaStatement> statements) throws EntityStorageException {
    +        requireNonNull(subject);
    +        requireNonNull(statements);
    +
    +        final MongoPcjDocuments pcjDocStore = pcjDocs.get();
    +        checkState(pcjDocStore != null, "Must set this indexers configuration before storing statements.");
    --- End diff --
    
    use checkNotNull


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r132279789
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    +     * @param accCon
    +     *            - connection to a valid Accumulo instance
    +     * @param tablename
    +     *            - name of an existing PCJ table
    +     * @throws PCJStorageException
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename)
    +            throws PCJStorageException, MalformedQueryException {
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        pcjDocs = indexer.getPcjStorage(conf);
    +        this.tablename = tablename;
    +    }
    +
    +    /**
    +     * returns size of table for query planning
    +     */
    +    @Override
    +    public double cardinality() {
    +        double cardinality = 0;
    +        try {
    +            cardinality = pcjDocs.getPcjMetadata(tablename).getCardinality();
    +        } catch (final PcjException e) {
    +            e.printStackTrace();
    +        }
    +        return cardinality;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindingset)
    +            throws QueryEvaluationException {
    +        return this.evaluate(Collections.singleton(bindingset));
    +    }
    +
    +    /**
    +     * Core evaluation method used during query evaluation - given a collection
    +     * of binding set constraints, this method finds common binding labels
    +     * between the constraints and table, uses those to build a prefix scan of
    +     * the Accumulo table, and creates a solution binding set by iterating of
    +     * the scan results.
    +     *
    +     * @param bindingset
    +     *            - collection of {@link BindingSet}s to be joined with PCJ
    +     * @return - CloseableIteration over joined results
    +     */
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final Collection<BindingSet> bindingset)
    +            throws QueryEvaluationException {
    +
    +        if (bindingset.isEmpty()) {
    +            return new IteratorWrapper<BindingSet, QueryEvaluationException>(new HashSet<BindingSet>().iterator());
    +        }
    +        final CloseableIterator<BindingSet> iter = pcjDocs.getResults(tablename, new Authorizations(), bindingset);
    --- End diff --
    
    we talked about this, decided to hold on off Auths since we didn't have a way of applying them without an updater.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132189025
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    --- End diff --
    
    Preconditions


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132202883
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjAdapter.java ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.BsonArray;
    +import org.bson.BsonDocument;
    +import org.bson.BsonString;
    +import org.bson.BsonValue;
    +import org.bson.Document;
    +import org.bson.codecs.DocumentCodec;
    +import org.bson.codecs.configuration.CodecRegistries;
    +import org.bson.codecs.configuration.CodecRegistry;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +/**
    + * Converts a Pcj for storage in mongoDB or retrieval from mongoDB.
    + */
    +public class MongoPcjAdapter implements BindingSetConverter<Bson> {
    --- End diff --
    
    Where is this class used?  It doesn't appear that this class or the converter above are used anywhere.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132201588
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoVisibilityBindingSetBsonConverter.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.DOCUMENT_VISIBILITY_KEY;
    +
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.MalformedDocumentVisibilityException;
    +import org.bson.BsonArray;
    +import org.bson.BsonDocument;
    +import org.bson.BsonString;
    +import org.bson.Document;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.impl.MapBindingSet;
    +
    +import com.mongodb.DBObject;
    +import com.mongodb.MongoClient;
    +import com.mongodb.util.JSON;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Converts {@link BindingSet}s to Strings and back again. The Strings do not
    + * include the binding names and are ordered with a {@link VariableOrder}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoVisibilityBindingSetBsonConverter {/* implements MongoBindingSetConverter {
    --- End diff --
    
    What's going on here?  Why is this commented out?


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160454100
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,172 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private static final Logger log = Logger.getLogger(PcjQueryNode.class);
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     * Creates a new {@link PcjQueryNode}.
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     *
    +     * @throws MalformedQueryException - The SPARQL query needs to contain a projection.
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs) throws MalformedQueryException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     * Creates a new {@link PcjQueryNode}.
    +     *
    +     * @param accCon - connection to a valid Accumulo instance
    +     * @param tablename - name of an existing PCJ table
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename) {
    --- End diff --
    
    @params don't match


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/655/<h2>Build result: FAILURE</span></h2>[...truncated 2.29 MB...][INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 14:49 min[INFO] Finished at: 2018-01-11T00:32:54+00:00[INFO] Final Memory: 227M/3308M[INFO] ------------------------------------------------------------------------Waiting for Jenkins to finish collecting data[ERROR] Failed to execute goal org.codehaus.mojo:findbugs-maven-plugin:3.0.5:findbugs (findbugs) on project rya.reasoning: Execution findbugs of goal org.codehaus.mojo:findbugs-maven-plugin:3.0.5:findbugs failed: Java returned: 134 -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run
  Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/PluginExecutionException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.reasoningchannel stoppedSetting status of 194dbd3e2c72f7dfb8daefe91cceb9b45bbe29d1 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/655/ and message: 'FAILURE 'Using context: Jenkins: clean package -Pgeoindexing



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160541792
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoCreatePCJ.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Set;
    +
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Sets;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoCreatePCJ implements CreatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoCreatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    --- End diff --
    
    This field is not used.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132020770
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    --- End diff --
    
    Yeah, I'm with Puja here.  A PcjIndex shouldn't deal with RyaStatements.  It should instead deal with BindingSets.  Is this meant to be the API for the eventual MongoPcjUpdater?


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160495312
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    +
    +        final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
    +            .collect(groupingBy(RyaStatement::getSubject));
    +
    +        for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
    +            try {
    +                updateEntity(entry.getKey(), entry.getValue());
    +            } catch (final EntityStorageException e) {
    +                throw new IOException("Failed to update the Entity index.", e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Updates a {@link Entity} to reflect new {@link RyaStatement}s.
    +     *
    +     * @param subject - The Subject of the {@link Entity} the statements are for. (not null)
    +     * @param statements - Statements that the {@link Entity} will be updated with. (not null)
    +     */
    +    private void updateEntity(final RyaURI subject, final Collection<RyaStatement> statements) throws EntityStorageException {
    +        requireNonNull(subject);
    +        requireNonNull(statements);
    +
    +        final MongoPcjDocuments pcjDocStore = pcjDocs.get();
    +        checkState(pcjDocStore != null, "Must set this indexers configuration before storing statements.");
    --- End diff --
    
    I thought I deleted these.....


---

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/250/<h2>Build result: FAILURE</span></h2>[...truncated 4.36 MB...][INFO] Apache Rya Accumulo Pig ............................ SKIPPED[INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 06:08 min[INFO] Finished at: 2017-07-06T00:00:52+00:00[INFO] Final Memory: 82M/936M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.indexing.pcj: Too many files with unapproved license: 3 See RAT report in: /home/jenkins/jenkins-slave/workspace/i
 ncubator-rya-master-with-optionals-pull-requests/extras/rya.indexing.pcj/target/rat.txt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.indexing.pcjchannel stoppedSetting status of aa642da65c7d06b597e949f1f4f1bf050e11062d to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/250/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528632
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    --- End diff --
    
    remove Accumulo exceptions


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

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534831
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    --- End diff --
    
    a lot of these look like I did them, its just a repackage


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160562616
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    --- End diff --
    
    probably should


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160811345
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/RyaDetailsFormatter.java ---
    @@ -72,30 +72,28 @@ public String format(StorageType storageType, final RyaDetails details) {
             report.append("  Temporal Index:\n");
             report.append("    Enabled: ").append( details.getTemporalIndexDetails().isEnabled() ).append("\n");
     
    -        if(storageType == StorageType.ACCUMULO) {
    -            report.append("  PCJ Index:\n");
    -            final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -            report.append("    Enabled: ").append( pcjDetails.isEnabled() ).append("\n");
    -            if(pcjDetails.isEnabled()) {
    -                if(pcjDetails.getFluoDetails().isPresent()) {
    -                    final String fluoAppName = pcjDetails.getFluoDetails().get().getUpdateAppName();
    -                    report.append("    Fluo App Name: ").append(fluoAppName).append("\n");
    -                }
    +        report.append("  PCJ Index:\n");
    --- End diff --
    
    that include Join Selectivity?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160790945
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    --- End diff --
    
    Why not just call this the PcjMetadataID where it is [pcj_id]_METADATA. I'm pretty sure the pcj name here is synonymous with the pcj id everywhere else in the code. This will be really confusing.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160549265
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(AUTHS_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjTableName - The name of the PCJ table that will receive the results. (not null)
    +     * @param ryaConn - A connection to the Rya store that will be queried to find results. (not null)
    +     * @throws PCJStorageException If results could not be written to the PCJ table,
    +     *   the PCJ table does not exist, or the query that is being execute
    +     *   was malformed.
    +     */
    +    public void populatePcj(final String pcjTableName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjTableName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                System.out.println(bs.toString());
    +                if(batch.size() == 1000) {
    +                    addResults(pcjTableName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjTableName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException("Could not populate a PCJ document with Rya results for the pcj named: " + pcjTableName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // get all results based on pcjName
    +        return queryForBindings(new Document(PCJ_NAME, pcjName));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjName.
    +     *
    +     * @param pcjName - The pcj to retrieve results for.
    +     * @param authorizations - The authorizations of the user to restrict results.
    +     * @param bindingset - The collection of {@link BindingSet}s to restrict results.
    --- End diff --
    
    Maybe call this restrictions or something?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160514341
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoCreatePCJ.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +import java.util.Set;
    +
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoCreatePCJ implements CreatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoCreatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoCreatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoCreatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public String createPCJ(final String ryaInstanceName, final String sparql, final Set<ExportStrategy> strategies) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(sparql);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    +        final MongoCredential cred;
    +        final MongoClient client;
    +
    +        final ServerAddress addr = new ServerAddress(connectionDetails.getHostname(), connectionDetails.getPort());
    +        if(username.isPresent()) {
    +            cred = MongoCredential.createCredential(username.get(), ryaInstanceName, connectionDetails.getPassword().get());
    +            client = new MongoClient(addr, Lists.newArrayList(cred));
    +        } else {
    +            client = new MongoClient(addr);
    +        }
    +
    +
    +        //pcjStore doesn't need to be closed since it doesn't need to release any resources.
    --- End diff --
    
    You could just throw it into a try with resources anyway since the interface suggests it may close resources. This code will be broken if for some reason the Mongo equivalent is updated to require a close call.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159803093
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjIndexer.java ---
    @@ -0,0 +1,38 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +
    +import edu.umd.cs.findbugs.annotations.Nullable;
    +
    +/**
    + * Updates the PCJs that are in a {@link MongoPcjDocuments}.
    + */
    +public interface PcjIndexer extends RyaSecondaryIndexer {
    +    /**
    +     * Creates the {@link MongoPcjDocuments} that will be used by the indexer.
    +     *
    +     * @param conf - Indicates how the {@link MongoPcjDocuments} is initialized. (not null)
    +     * @return The {@link MongoPcjDocuments} that will be used by this indexer.
    +     */
    +    public @Nullable MongoPcjDocuments getPcjStorage(Configuration conf);
    --- End diff --
    
    Shouldn't this method be getPcjDocuments?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160539217
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    --- End diff --
    
    Should this be makeMetadataDocument(...)? It seems to just be a factory method.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/627/<h2>Failed Tests: <span class='status-failure'>2</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:mongodb.rya' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/627/org.apache.rya$mongodb.rya/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:mongodb.rya</a>: <span class='status-failure'>2</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/627/org.apache.rya$mongodb.rya/testReport/org.apache.rya.mongodb.instance/MongoDetailsAdapterTest/ryaDetailsToMongoTest/'><strong>org.apache.rya.mongodb.instance.MongoDetailsAdapterTest.ryaDetailsToMongoTest</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/627/org.apache.rya$mongodb.rya/testReport/org.apache.rya.mongodb.instance/MongoDetails
 AdapterTest/absentOptionalToMongoTest/'><strong>org.apache.rya.mongodb.instance.MongoDetailsAdapterTest.absentOptionalToMongoTest</strong></a></li></ul>



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160527895
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            final MongoClient client = mongoConf.getMongoClient();
    +            final MongoPcjDocuments pcjDocs = new MongoPcjDocuments(client, mongoConf.getRyaInstanceName());
    +            List<String> tables = null;
    --- End diff --
    
    Are the tables actually collections? Should we call them that?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132175735
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,125 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private final MongoClient client;
    +    private final MongoDBRdfConfiguration mongoConf;
    +
    +    public MongoPcjIndexSetProvider(final Configuration conf, final MongoClient client) {
    +        super(conf);
    +        this.client = client;
    +        mongoConf = new MongoDBRdfConfiguration(conf);
    +    }
    +
    +    public MongoPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices, final MongoClient client) {
    +        super(conf, indices);
    +        this.client = client;
    --- End diff --
    
    Preconditions


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532120
  
    --- Diff: extras/rya.benchmark/src/main/gen/org/apache/rya/benchmark/query/QueriesBenchmarkConf.java ---
    @@ -20,7 +20,7 @@
     // This file was generated by the JavaTM Architecture for XML Binding(JAXB) Reference Implementation, v2.2.11 
     // See <a href="http://java.sun.com/xml/jaxb">http://java.sun.com/xml/jaxb</a> 
     // Any modifications to this file will be lost upon recompilation of the source schema. 
    -// Generated on: 2016.12.16 at 01:22:14 PM PST 
    +// Generated on: 2017.07.06 at 03:13:11 PM EDT 
    --- End diff --
    
    revert


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139544146
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    --- End diff --
    
    This was a copy from the Accumulo PrecomputerJoinIndexer


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160760282
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoCreatePCJ.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Set;
    +
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Sets;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoCreatePCJ implements CreatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoCreatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    --- End diff --
    
    removed


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159925371
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    --- End diff --
    
    removed


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160540004
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    --- End diff --
    
    [value]


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132193846
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizer.java ---
    @@ -90,9 +97,19 @@ public final void setConf(final Configuration conf) {
             if (!init) {
                 try {
                     this.conf = conf;
    -                this.useOptimal = ConfigUtils.getUseOptimalPCJ(conf);
    -                provider = new AccumuloIndexSetProvider(conf);
    -            } catch (Exception e) {
    +                useOptimal = ConfigUtils.getUseOptimalPCJ(conf);
    --- End diff --
    
    Ugh, I hate that we have to do this and we can't use an Interface.  Stupid setConf() init.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532578
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    public Document getMetadataDocument(final String pcjName, final String sparql) {
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn
    +     *            - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName
    +     *            - The name of the PCJ table that will be created. (not null)
    +     * @param sparql
    +     *            - The SPARQL query whose results will be loaded into the
    +     *            table. (not null)
    +     * @throws PCJStorageException
    +     *             The PCJ table could not be create or the values from Rya were
    +     *             not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, new ArrayList<VariableOrder>());
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                            .append(BINDING_TYPE, type.getDataType().stringValue())
    +                            .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(AUTHS_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, String.format("/%s/i", pcjName));
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjTableName - The name of the PCJ table that will receive the results. (not null)
    +     * @param ryaConn - A connection to the Rya store that will be queried to find results. (not null)
    +     * @throws PCJStorageException If results could not be written to the PCJ table,
    +     *   the PCJ table does not exist, or the query that is being execute
    +     *   was malformed.
    +     */
    +    public void populatePcj(final String pcjTableName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjTableName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                System.out.println(bs.toString());
    +                if(batch.size() == 1000) {
    +                    addResults(pcjTableName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjTableName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException("Could not populate a PCJ document with Rya results for the pcj named: " + pcjTableName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +            //{} - no search criteria: find all
    +            //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().getString(PCJ_ID).replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName
    --- End diff --
    
    nit: fix whitepace


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160229044
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -141,7 +174,8 @@ public boolean arePCJCommandsAvailable() {
             // The PCJ commands are only available if the Shell is connected to an instance of Rya
             // that is new enough to use the RyaDetailsRepository and is configured to maintain PCJs.
             final ShellState shellState = state.getShellState();
    -        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE) {
    +        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE &&
    +                shellState.getStorageType().get() == StorageType.ACCUMULO) {
    --- End diff --
    
    Check RyaAdminCommandsTest in the rya.shell project.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160532572
  
    --- Diff: extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java ---
    @@ -41,7 +41,7 @@
      */
     public class MongoInstallIT extends MongoITBase {
     
    -    @Test
    +	@Test
    --- End diff --
    
    tab


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761434
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(AUTHS_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjTableName - The name of the PCJ table that will receive the results. (not null)
    +     * @param ryaConn - A connection to the Rya store that will be queried to find results. (not null)
    +     * @throws PCJStorageException If results could not be written to the PCJ table,
    +     *   the PCJ table does not exist, or the query that is being execute
    +     *   was malformed.
    +     */
    +    public void populatePcj(final String pcjTableName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjTableName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                System.out.println(bs.toString());
    --- End diff --
    
    removed


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160755315
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    --- End diff --
    
    The way VisibilityBindingSets were implemented, I think they use accumulo's Authorizations api


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159798308
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    +                        result = (VisibilityBindingSet) bindingSet;
    +                    } else {
    +                        //TODO warn that visibilities are being lost.  do we want to exit?
    +                        log.warn("some message");
    --- End diff --
    
    Visibilities will always be lost for a batch update.  This is something that should probably be documented at the class level as opposed to in an error message.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159754654
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    --- End diff --
    
    Mongo Implementation of BatchUpdatePCJ


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534892
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjAdapter.java ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.BsonArray;
    +import org.bson.BsonDocument;
    +import org.bson.BsonString;
    +import org.bson.BsonValue;
    +import org.bson.Document;
    +import org.bson.codecs.DocumentCodec;
    +import org.bson.codecs.configuration.CodecRegistries;
    +import org.bson.codecs.configuration.CodecRegistry;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.algebra.evaluation.QueryBindingSet;
    +
    +/**
    + * Converts a Pcj for storage in mongoDB or retrieval from mongoDB.
    + */
    +public class MongoPcjAdapter implements BindingSetConverter<Bson> {
    --- End diff --
    
    This class is still present.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160528789
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            final MongoClient client = mongoConf.getMongoClient();
    +            final MongoPcjDocuments pcjDocs = new MongoPcjDocuments(client, mongoConf.getRyaInstanceName());
    +            List<String> tables = null;
    --- End diff --
    
    Oh, this is legacy interface support and the table name is a leftover from accumulo. This is how we let you provide a list of collection names that represent the pcj indices through configuration instead of asking details. Could you document that is what's going on?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534554
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    --- End diff --
    
    We should call these visibilities instead of auths since the two are very different things. Visiblities define who is able to access a binding set. It is an expression such as "(a&b)|(c&d)". Auths are a list of labels a user has which are evaluated agains the visibility expressions, such as "a, b, c". We label data with visibility expressions. Users use auths to access that data.


---

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/247/<h2>Build result: FAILURE</span></h2>[...truncated 32.44 KB...][INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 32.897 s[INFO] Finished at: 2017-07-05T15:09:34+00:00[INFO] Final Memory: 41M/553M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.api: Too many files with unapproved license: 1 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/common/rya.api/target/rat.t
 xt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.apichannel stoppedSetting status of f8a8d2701366519e36a6cc79a2156f0cb38f71b0 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/247/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160813231
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    --- End diff --
    
    you store the PCJ ID plus metadata


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159922006
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjIndexer.java ---
    @@ -0,0 +1,38 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +
    +import edu.umd.cs.findbugs.annotations.Nullable;
    +
    +/**
    + * Updates the PCJs that are in a {@link MongoPcjDocuments}.
    + */
    +public interface PcjIndexer extends RyaSecondaryIndexer {
    +    /**
    +     * Creates the {@link MongoPcjDocuments} that will be used by the indexer.
    +     *
    +     * @param conf - Indicates how the {@link MongoPcjDocuments} is initialized. (not null)
    +     * @return The {@link MongoPcjDocuments} that will be used by this indexer.
    +     */
    +    public @Nullable MongoPcjDocuments getPcjStorage(Configuration conf);
    --- End diff --
    
    yes


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532311
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/AbstractPcjStorage.java ---
    @@ -0,0 +1,81 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +
    +public abstract class AbstractPcjStorage implements PrecomputedJoinStorage {
    +    // Used to update the instance's metadata.
    +    protected final RyaDetailsRepository ryaDetailsRepo;
    +
    +    protected final String ryaInstanceName;
    +
    +    // Factories that are used to create new PCJs.
    +    protected final PCJIdFactory pcjIdFactory = new PCJIdFactory();
    +    protected final PcjTableNameFactory pcjTableNameFactory = new PcjTableNameFactory();
    +    protected final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    public AbstractPcjStorage(final RyaDetailsRepository ryaDetailsRepo, final String ryaInstanceName) {
    --- End diff --
    
    doc


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

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/248/<h2>Build result: FAILURE</span></h2>[...truncated 32.43 KB...][INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 28.411 s[INFO] Finished at: 2017-07-05T18:49:32+00:00[INFO] Final Memory: 44M/623M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.api: Too many files with unapproved license: 1 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/common/rya.api/target/rat.t
 xt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.apichannel stoppedSetting status of a16c5f204639d4f6736769c21fed7cd8daaeb52b to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/248/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160546531
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    --- End diff --
    
    left in [Mongo].


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160813158
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    --- End diff --
    
    it lets mongo handle that


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831446
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    +     * @param sparql
    +     *            - The query the pcj is assigned to.
    +     * @throws @throws
    +     *             PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjId - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjId) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjId))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjId + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * 
    +     * @param pcjId
    +     *            - The PCJ to add the results to.
    +     * @param results
    +     *            - The binding set results.
    +     */
    +    public void addResults(final String pcjId, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjId);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjId);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ document to purge. (not null)
    --- End diff --
    
    The [ID] of the...


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160793949
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjStorage.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PCJIdFactory;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.BindingSet;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A mongo backed implementation of {@link PrecomputedJoinStorage}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPcjStorage implements PrecomputedJoinStorage {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +    // Used to update the instance's metadata.
    +    private final MongoRyaInstanceDetailsRepository ryaDetailsRepo;
    +
    +    private final String ryaInstanceName;
    +
    +    // Factories that are used to create new PCJs.
    +    private final PCJIdFactory pcjIdFactory = new PCJIdFactory();
    +
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     * Constructs an instance of {@link MongoPcjStorage}.
    +     *
    +     * @param client - The {@link MongoClient} that will be used to connect to Mongodb. (not null)
    +     * @param ryaInstanceName - The name of the RYA instance that will be accessed. (not null)
    +     */
    +    public MongoPcjStorage(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        this.ryaInstanceName = requireNonNull(ryaInstanceName);
    +        pcjDocs = new MongoPcjDocuments(client, ryaInstanceName);
    +        ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +    }
    +
    +    @Override
    +    public String createPcj(final String sparql) throws PCJStorageException {
    +        requireNonNull(sparql);
    +
    +        // Update the Rya Details for this instance to include the new PCJ
    +        // table.
    +        final String pcjId = pcjIdFactory.nextId();
    +
    +        try {
    +            new RyaDetailsUpdater(ryaDetailsRepo).update(originalDetails -> {
    +                // Create the new PCJ's details.
    +                final PCJDetails.Builder newPcjDetails = PCJDetails.builder().setId(pcjId);
    +
    +                // Add them to the instance's details.
    +                final RyaDetails.Builder mutated = RyaDetails.builder(originalDetails);
    +                mutated.getPCJIndexDetails().addPCJDetails(newPcjDetails);
    +                return mutated.build();
    +            });
    +        } catch (final RyaDetailsRepositoryException | CouldNotApplyMutationException e) {
    +            throw new PCJStorageException(String.format("Could not create a new PCJ for Rya instance '%s' "
    +                    + "because of a problem while updating the instance's details.", ryaInstanceName), e);
    +        }
    +
    +        // Create the objectID of the document to house the PCJ results.
    +        pcjDocs.createPcj(pcjId, sparql);
    --- End diff --
    
    See, right here you're calling it a pcjId which is in line with the rest of the application, but then inside of that class you're calling it a pcjName and changing the meaning of pcjId. That's inconsistent.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534139
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoInstall.java ---
    @@ -147,22 +147,23 @@ private RyaDetails initializeRyaDetails(
                 final InstallConfiguration installConfig) throws AlreadyInitializedException, RyaDetailsRepositoryException {
             final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(adminClient, instanceName);
     
    -        // Build the PCJ Index details. [not supported in mongo]
    -        final PCJIndexDetails.Builder pcjDetailsBuilder = PCJIndexDetails.builder().setEnabled(false);
    +        if(installConfig.getFluoPcjAppName().isPresent()) {
    +        	log.warn("Mongo does not have fluo support, use ignoring the configured fluo application name: " + installConfig.getFluoPcjAppName().get());
    +        }
    +        
    +        // Build the PCJ Index details.
    +        final PCJIndexDetails.Builder pcjDetailsBuilder = PCJIndexDetails.builder()
    +                .setEnabled(installConfig.isPcjIndexEnabled());
     
             final RyaDetails details = RyaDetails.builder()
                     // General Metadata
                     .setRyaInstanceName(instanceName).setRyaVersion(getVersion())
     
    -                // FIXME RYA-215 .setGeoIndexDetails(new GeoIndexDetails(installConfig.isGeoIndexEnabled()))
    -
                     // Secondary Index Values
    +                // FIXME RYA-215 .setGeoIndexDetails(new GeoIndexDetails(installConfig.isGeoIndexEnabled()))
                     .setTemporalIndexDetails(new TemporalIndexDetails(installConfig.isTemporalIndexEnabled()))
                     .setFreeTextDetails(new FreeTextIndexDetails(installConfig.isFreeTextIndexEnabled()))//
    -
    -                // Entity centric indexing is not supported in Mongo DB.
    -                .setEntityCentricIndexDetails(new EntityCentricIndexDetails(false))
    -
    +                .setEntityCentricIndexDetails(new EntityCentricIndexDetails(installConfig.isEntityCentrixIndexEnabled()))
    --- End diff --
    
    discussed in person


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570534
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    no, the id gets calculated based on the name


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160548641
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    --- End diff --
    
    Do we want to mix Accumulo specific code with the Mongo code, even though it's a holder for Authorizations?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160810948
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    --- End diff --
    
    I mean how @ejwhite922 initially implemented it



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570420
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    --- End diff --
    
    no, its for the pcj name.  the ID is calculated differently


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160536863
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    --- End diff --
    
    visibilities field


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160538868
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    There are all the ID of the PCJ, right?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160529512
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            final MongoClient client = mongoConf.getMongoClient();
    +            final MongoPcjDocuments pcjDocs = new MongoPcjDocuments(client, mongoConf.getRyaInstanceName());
    +            List<String> tables = null;
    --- End diff --
    
    Or should the abstract class handle that case for you and the abstract method just has to fetch them from the appropriate RyaDetails repository? Could this whole thing be handled by returning a RyaDetailsRepo that is connected to the configured rya instance?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    just remove the accumulo check from RyaDetailsFormatter


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160795755
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    --- End diff --
    
    Could you be more specific? You mean how we query Mongo DB for results? VisibilityBindingSet does not depend on anything in Accumulo.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160546683
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    --- End diff --
    
    null check this?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/642/<h2>Build result: FAILURE</span></h2>[...truncated 363.37 KB...][ERROR] location: class org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments[ERROR] /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java:[292,23] cannot find symbol[ERROR] symbol:   class VisibilityBindingSet[ERROR] location: class org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments[ERROR] /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java:[292,53] cannot find symbol[ERROR] symbol:   class VisibilityBindingSet[ERROR] location: class org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments[ERROR] /home/jenkins/jenkins-slave/workspace/incubator-r
 ya-master-with-optionals-pull-requests/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java:[397,20] cannot find symbol[ERROR] symbol:   class CloseableIterator[ERROR] location: class org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments[ERROR] -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.indexing.pcjchannel stoppedSetting status of bcf183755de2b8cdf8674e180eaf9f3cd1438a92 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/642/ and message: 'F
 AILURE 'Using context: Jenkins: clean package -Pgeoindexing



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160494372
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java ---
    @@ -51,17 +51,29 @@ public static void addRyaDetailsToConfiguration(final RyaDetails details, final
     
             checkAndSet(conf, ConfigurationFields.USE_ENTITY, details.getEntityCentricIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_FREETEXT, details.getFreeTextIndexDetails().isEnabled());
    -      //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
    +        //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_TEMPORAL, details.getTemporalIndexDetails().isEnabled());
             final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -        if (pcjDetails.isEnabled() && pcjDetails.getFluoDetails().isPresent()) {
    -            checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, true);
    -            conf.set(ConfigurationFields.FLUO_APP_NAME, pcjDetails.getFluoDetails().get().getUpdateAppName());
    -            conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "FLUO");
    -            conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "ACCUMULO");
    +        
    +        if(conf.getBoolean(ConfigurationFields.USE_MONGO, false) ) {
    +        	if(pcjDetails.isEnabled()) {
    +        		conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "MONGO");
    --- End diff --
    
    there's no scope to it from rya.api


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160532307
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    +                        result = (VisibilityBindingSet) bindingSet;
    +                    } else {
    +                        //TODO warn that visibilities are being lost.  do we want to exit?
    +                        log.warn("some message");
    --- End diff --
    
    again, this was all deleted


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160298620
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    --- End diff --
    
    This might be clearer if you say "..." instead of having each dot on a separate line.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127530560
  
    --- Diff: extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java ---
    @@ -379,15 +378,16 @@ public void testEvaluateTwoIndexUnionFilter() throws AccumuloException,
     
     		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
     		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
    -		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
    +		repo = PcjIntegrationTestingUtil.getAccumuloPcjRepo(tablePrefix, "instance");
     		conn = repo.getConnection();
     		conn.add(sub2, RDF.TYPE, subclass2);
     		conn.add(sub2, RDF.TYPE, obj2);
     		final CountingResultHandler crh = new CountingResultHandler();
     		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
     				crh);
     
    -		Assert.assertEquals(6, crh.getCount());
    +        // THIS WAS 6, but why?
    --- End diff --
    
    Address this


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160796820
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjName);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ
    +     * document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjName
    +     *            - The name of the PCJ table that will receive the results. (not
    +     *            null)
    +     * @param ryaConn
    +     *            - A connection to the Rya store that will be queried to find
    +     *            results. (not null)
    +     * @throws PCJStorageException
    +     *             If results could not be written to the PCJ table, the PCJ table
    +     *             does not exist, or the query that is being execute was malformed.
    +     */
    +    public void populatePcj(final String pcjName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                if(batch.size() == 1000) {
    +                    addResults(pcjName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException(
    +                    "Could not populate a PCJ document with Rya results for the pcj named: " + pcjName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // get all results based on pcjName
    +        return queryForBindings(new Document(PCJ_NAME, pcjName));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjName.
    +     *
    +     * @param pcjName
    +     *            - The pcj to retrieve results for.
    +     * @param authorizations
    +     *            - The authorizations of the user to restrict results.
    +     * @param restrictionBindings
    +     *            - The collection of {@link BindingSet}s to restrict results.
    +     *            <p>
    +     *            Note: the result restrictions from {@link BindingSet}s are an OR
    +     *            over ANDS in that: <code>
    +     *  [
    +     *     bindingset: binding AND binding AND binding,
    +     *     OR
    +     *     bindingset: binding AND binding AND binding,
    +     *     .
    +     *     .
    +     *     .
    +     *     OR
    +     *     bindingset: binding
    +     *  ]
    +     * </code>
    +     * @return
    +     */
    +    public CloseableIterator<BindingSet> getResults(final String pcjName, final Authorizations authorizations,
    --- End diff --
    
    The authorizations field isn't being used within this method.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831244
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    --- End diff --
    
    Update docs. Uniquely identifies a PCJ within Rya.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159978264
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/RyaAdminCommands.java ---
    @@ -141,7 +174,8 @@ public boolean arePCJCommandsAvailable() {
             // The PCJ commands are only available if the Shell is connected to an instance of Rya
             // that is new enough to use the RyaDetailsRepository and is configured to maintain PCJs.
             final ShellState shellState = state.getShellState();
    -        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE) {
    +        if(shellState.getConnectionState() == ConnectionState.CONNECTED_TO_INSTANCE &&
    +                shellState.getStorageType().get() == StorageType.ACCUMULO) {
    --- End diff --
    
    Is this review supposed to include shell updates? Because if so, that work does not appear to be done yet. I'd encourage that work being a separate ticket.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160517028
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    --- End diff --
    
    I have a feeling this entire class can be deleted and it will not effect anything.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160534078
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,96 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoDeletePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoDeletePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoDeletePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public void deletePCJ(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160830222
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjName);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ
    +     * document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjName
    +     *            - The name of the PCJ table that will receive the results. (not
    +     *            null)
    +     * @param ryaConn
    +     *            - A connection to the Rya store that will be queried to find
    +     *            results. (not null)
    +     * @throws PCJStorageException
    +     *             If results could not be written to the PCJ table, the PCJ table
    +     *             does not exist, or the query that is being execute was malformed.
    +     */
    +    public void populatePcj(final String pcjName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                if(batch.size() == 1000) {
    +                    addResults(pcjName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException(
    +                    "Could not populate a PCJ document with Rya results for the pcj named: " + pcjName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // get all results based on pcjName
    +        return queryForBindings(new Document(PCJ_NAME, pcjName));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjName.
    +     *
    +     * @param pcjName
    +     *            - The pcj to retrieve results for.
    +     * @param authorizations
    +     *            - The authorizations of the user to restrict results.
    +     * @param restrictionBindings
    +     *            - The collection of {@link BindingSet}s to restrict results.
    +     *            <p>
    +     *            Note: the result restrictions from {@link BindingSet}s are an OR
    +     *            over ANDS in that: <code>
    +     *  [
    +     *     bindingset: binding AND binding AND binding,
    +     *     OR
    +     *     bindingset: binding AND binding AND binding,
    +     *     .
    +     *     .
    +     *     .
    +     *     OR
    +     *     bindingset: binding
    +     *  ]
    +     * </code>
    +     * @return
    +     */
    +    public CloseableIterator<BindingSet> getResults(final String pcjName, final Authorizations authorizations,
    --- End diff --
    
    Then why does this method exist at all? It will never behave as it is documented.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831524
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    +     * @param sparql
    +     *            - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException
    +     *             - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     * 
    +     * @param pcjId
    +     *            - The unique name of the PCJ.
    +     * @param sparql
    +     *            - The query the pcj is assigned to.
    +     * @throws @throws
    +     *             PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjId - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjId) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjId))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjId + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * 
    +     * @param pcjId
    +     *            - The PCJ to add the results to.
    +     * @param results
    +     *            - The binding set results.
    +     */
    +    public void addResults(final String pcjId, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjId);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjId);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjId) {
    +        requireNonNull(pcjId);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjId);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ
    +     * document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjId
    +     *            - The name of the PCJ table that will receive the results. (not
    --- End diff --
    
    I think everywhere this class mentions "PCJ table" that it is copy paste from the Accumulo implementation. It doesn't describe how it works in Mongo DB. This whole class could use a second pass to ensure it's documented with the Mongo implementation in mind.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160513515
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    +                        result = (VisibilityBindingSet) bindingSet;
    +                    } else {
    +                        //TODO warn that visibilities are being lost.  do we want to exit?
    +                        log.warn("some message");
    --- End diff --
    
    Address the TODO.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160833809
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528902
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    +     * @param accCon
    +     *            - connection to a valid Accumulo instance
    +     * @param tablename
    +     *            - name of an existing PCJ table
    +     * @throws PCJStorageException
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename)
    +            throws PCJStorageException, MalformedQueryException {
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        pcjDocs = indexer.getPcjStorage(conf);
    +        this.tablename = tablename;
    +    }
    +
    +    /**
    +     * returns size of table for query planning
    +     */
    +    @Override
    +    public double cardinality() {
    +        double cardinality = 0;
    +        try {
    +            cardinality = pcjDocs.getPcjMetadata(tablename).getCardinality();
    +        } catch (final PcjException e) {
    +            e.printStackTrace();
    +        }
    +        return cardinality;
    +    }
    +
    +    @Override
    +    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindingset)
    +            throws QueryEvaluationException {
    +        return this.evaluate(Collections.singleton(bindingset));
    +    }
    +
    +    /**
    +     * Core evaluation method used during query evaluation - given a collection
    +     * of binding set constraints, this method finds common binding labels
    +     * between the constraints and table, uses those to build a prefix scan of
    +     * the Accumulo table, and creates a solution binding set by iterating of
    +     * the scan results.
    +     *
    +     * @param bindingset
    --- End diff --
    
    nit: adjust


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159803931
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java ---
    @@ -1,221 +0,0 @@
    -/**
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - *     http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing,
    - * software distributed under the License is distributed on an
    - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    - * KIND, either express or implied.  See the License for the
    - * specific language governing permissions and limitations
    - * under the License.
    - */
    -package org.apache.rya.indexing.pcj.matching;
    -
    -import java.util.ArrayList;
    -import java.util.Iterator;
    -import java.util.List;
    -import java.util.Map;
    -import java.util.Objects;
    -
    -import org.apache.accumulo.core.client.AccumuloException;
    -import org.apache.accumulo.core.client.AccumuloSecurityException;
    -import org.apache.accumulo.core.client.Connector;
    -import org.apache.accumulo.core.client.TableNotFoundException;
    -import org.apache.hadoop.conf.Configuration;
    -import org.apache.log4j.Logger;
    -import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
    -import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    -import org.apache.rya.api.instance.RyaDetailsRepository;
    -import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    -import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    -import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    -import org.apache.rya.indexing.accumulo.ConfigUtils;
    -import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    -import org.apache.rya.indexing.external.matching.QuerySegment;
    -import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
    -import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    -import org.apache.rya.indexing.pcj.storage.PcjException;
    -import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    -import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
    -import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    -import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
    -import org.openrdf.query.MalformedQueryException;
    -import org.openrdf.query.QueryEvaluationException;
    -import org.openrdf.query.algebra.TupleExpr;
    -import org.openrdf.sail.SailException;
    -
    -import com.google.common.base.Preconditions;
    -import com.google.common.collect.Lists;
    -import com.google.common.collect.Maps;
    -
    -/**
    - * Implementation of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    - * This provider uses either user specified Accumulo configuration information or user a specified
    - * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If Accumulo configuration
    - * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    - * PCJs registered in RyaDetails.
    - *
    - */
    -public class AccumuloIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    --- End diff --
    
    Why are you deleting anything related to Accumulo?  What was this replaced by?
      


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160788899
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/RyaDetailsFormatter.java ---
    @@ -72,30 +72,28 @@ public String format(StorageType storageType, final RyaDetails details) {
             report.append("  Temporal Index:\n");
             report.append("    Enabled: ").append( details.getTemporalIndexDetails().isEnabled() ).append("\n");
     
    -        if(storageType == StorageType.ACCUMULO) {
    -            report.append("  PCJ Index:\n");
    -            final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -            report.append("    Enabled: ").append( pcjDetails.isEnabled() ).append("\n");
    -            if(pcjDetails.isEnabled()) {
    -                if(pcjDetails.getFluoDetails().isPresent()) {
    -                    final String fluoAppName = pcjDetails.getFluoDetails().get().getUpdateAppName();
    -                    report.append("    Fluo App Name: ").append(fluoAppName).append("\n");
    -                }
    +        report.append("  PCJ Index:\n");
    --- End diff --
    
    The Statistics section does not have anything to do with PCJs so it shouldn't be printed conditionally when PCJ is turned on. They are also an Accumulo only feature.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160518640
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     *
    +     * @return Iterator of Lists (combos) of PCJs used to build an optimal query plan
    +     */
    +    @Override
    +    public Iterator<List<ExternalTupleSet>> getExternalSetCombos(final QuerySegment<ExternalTupleSet> segment) {
    +        final ValidIndexCombinationGenerator comboGen = new ValidIndexCombinationGenerator(segment.getOrderedNodes());
    +        return comboGen.getValidIndexCombos(getExternalSets(segment));
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     * @return List of PCJs for matching
    +     */
    +    @Override
    +    public List<ExternalTupleSet> getExternalSets(final QuerySegment<ExternalTupleSet> segment) {
    +        try {
    +            if(!init) {
    +                indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +                init = true;
    +            }
    +            final TupleExpr query = segment.getQuery().getTupleExpr();
    +            final IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(query, indexCache);
    +            final List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
    +            final List<ExternalTupleSet> tuples = new ArrayList<>();
    +            for (final ExternalTupleSet tuple: pcjs) {
    +                final QuerySegment<ExternalTupleSet> pcj = converter.setToSegment(tuple);
    +                if (segment.containsQuerySegment(pcj)) {
    +                    tuples.add(tuple);
    +                }
    +            }
    +            return tuples;
    +
    +        } catch (final Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    /**
    +     * @return The size of the set index cache.
    +     * @throws Exception
    --- End diff --
    
    Document the exception.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160813417
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, makeMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, makeMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjName);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, makeMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ
    +     * document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjName
    +     *            - The name of the PCJ table that will receive the results. (not
    +     *            null)
    +     * @param ryaConn
    +     *            - A connection to the Rya store that will be queried to find
    +     *            results. (not null)
    +     * @throws PCJStorageException
    +     *             If results could not be written to the PCJ table, the PCJ table
    +     *             does not exist, or the query that is being execute was malformed.
    +     */
    +    public void populatePcj(final String pcjName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                if(batch.size() == 1000) {
    +                    addResults(pcjName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException(
    +                    "Could not populate a PCJ document with Rya results for the pcj named: " + pcjName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // get all results based on pcjName
    +        return queryForBindings(new Document(PCJ_NAME, pcjName));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjName.
    +     *
    +     * @param pcjName
    +     *            - The pcj to retrieve results for.
    +     * @param authorizations
    +     *            - The authorizations of the user to restrict results.
    +     * @param restrictionBindings
    +     *            - The collection of {@link BindingSet}s to restrict results.
    +     *            <p>
    +     *            Note: the result restrictions from {@link BindingSet}s are an OR
    +     *            over ANDS in that: <code>
    +     *  [
    +     *     bindingset: binding AND binding AND binding,
    +     *     OR
    +     *     bindingset: binding AND binding AND binding,
    +     *     .
    +     *     .
    +     *     .
    +     *     OR
    +     *     bindingset: binding
    +     *  ]
    +     * </code>
    +     * @return
    +     */
    +    public CloseableIterator<BindingSet> getResults(final String pcjName, final Authorizations authorizations,
    --- End diff --
    
    because we're not sure how to use it right now.  


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160548138
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(AUTHS_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjTableName - The name of the PCJ table that will receive the results. (not null)
    +     * @param ryaConn - A connection to the Rya store that will be queried to find results. (not null)
    +     * @throws PCJStorageException If results could not be written to the PCJ table,
    +     *   the PCJ table does not exist, or the query that is being execute
    +     *   was malformed.
    +     */
    +    public void populatePcj(final String pcjTableName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjTableName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                System.out.println(bs.toString());
    --- End diff --
    
    SOP


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139547647
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    --- End diff --
    
    the indexer keeps the connection state for mongo.  If we want to break that out...I guess we can.  It goes against how we've done all the other indexers.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160792409
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    --- End diff --
    
    This field is kind of misleading since you don't actually store the PCJ ID in it.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160532183
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/instance/MongoDetailsAdapter.java ---
    @@ -124,11 +123,6 @@ private static DBObject toDBObject(final PCJIndexDetails pcjIndexDetails) {
             // Is Enabled
             builder.add(PCJ_ENABLED_KEY, pcjIndexDetails.isEnabled());
     
    -        // Fluo Details if present.
    -        if(pcjIndexDetails.getFluoDetails().isPresent()) {
    --- End diff --
    
    This seems like a bigger problem outside the scope of this.  I deleted it because tests were failing throughout installconfig expecting to see fluo stuff and it not being there.  Adding it back in would require changes to be made elsewhere to get glue code


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160518292
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    --- End diff --
    
    If this is going to be exposed for testing, could it at least be package scoped? Would prefer if this method did not exist. If it's going to stay, it should be documented.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127529537
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java ---
    @@ -375,4 +375,6 @@ public void meet(LeftJoin node) {
     
     
     
    +
    +
    --- End diff --
    
    nit: whitespace


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159801144
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    +
    +        final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
    +            .collect(groupingBy(RyaStatement::getSubject));
    +
    +        for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
    +            try {
    +                updateEntity(entry.getKey(), entry.getValue());
    +            } catch (final EntityStorageException e) {
    +                throw new IOException("Failed to update the Entity index.", e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Updates a {@link Entity} to reflect new {@link RyaStatement}s.
    +     *
    +     * @param subject - The Subject of the {@link Entity} the statements are for. (not null)
    +     * @param statements - Statements that the {@link Entity} will be updated with. (not null)
    +     */
    +    private void updateEntity(final RyaURI subject, final Collection<RyaStatement> statements) throws EntityStorageException {
    +        requireNonNull(subject);
    --- End diff --
    
    This method still seems off to me.  You're going to be adding statements to any updater that you need.  There is no need to include this method.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    what is RyaDetailsFormatter..... this is more of a problem of just stuff floating around that doesn't break when functionality gets added


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    I think RyaDetailsFormatter still needs to be updated for this review.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160830916
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    --- End diff --
    
    pcj name


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160575832
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    Why? 


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160453846
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    +
    +        final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
    +            .collect(groupingBy(RyaStatement::getSubject));
    +
    +        for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
    +            try {
    +                updateEntity(entry.getKey(), entry.getValue());
    +            } catch (final EntityStorageException e) {
    +                throw new IOException("Failed to update the Entity index.", e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Updates a {@link Entity} to reflect new {@link RyaStatement}s.
    +     *
    +     * @param subject - The Subject of the {@link Entity} the statements are for. (not null)
    +     * @param statements - Statements that the {@link Entity} will be updated with. (not null)
    +     */
    +    private void updateEntity(final RyaURI subject, final Collection<RyaStatement> statements) throws EntityStorageException {
    +        requireNonNull(subject);
    +        requireNonNull(statements);
    +
    +        final MongoPcjDocuments pcjDocStore = pcjDocs.get();
    +        checkState(pcjDocStore != null, "Must set this indexers configuration before storing statements.");
    +    }
    +
    +    @Override
    +    public void deleteStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +
    +        final MongoPcjDocuments pcjDocStore = pcjDocs.get();
    +        checkState(pcjDocStore != null, "Must set this indexers configuration before storing statements.");
    --- End diff --
    
    same


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159929210
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,51 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    +    private static final Logger log = Logger.getLogger(MongoPrecomputedJoinIndexer.class);
    +
    +    @Override
    +    public MongoPcjDocuments getPcjStorage(final Configuration conf) {
    --- End diff --
    
    no, it needs to return documents.  MongoPcjDocuments is the mongo equivalent of accumulo's PcjTables


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159798116
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJ.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Date;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.rya.api.client.BatchUpdatePCJ;
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.PCJDoesNotExistException;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
    +import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails.PCJUpdateStrategy;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.api.instance.RyaDetailsUpdater;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator;
    +import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
    +import org.apache.rya.sail.config.RyaSailFactory;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResultHandlerBase;
    +import org.openrdf.query.TupleQueryResultHandlerException;
    +import org.openrdf.repository.RepositoryException;
    +import org.openrdf.repository.sail.SailRepository;
    +import org.openrdf.repository.sail.SailRepositoryConnection;
    +import org.openrdf.sail.Sail;
    +import org.openrdf.sail.SailConnection;
    +import org.openrdf.sail.SailException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoBatchUpdatePCJ implements BatchUpdatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoBatchUpdatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +    private final MongoClient mongoClient;
    +
    +    /**
    +     * Constructs an instance of {@link MongoBatchUpdatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     * @param mongoClient - The {@link MongoClient} to use when batch updating. (not null)
    +     */
    +    public MongoBatchUpdatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists,
    +            final MongoClient mongoClient) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +        this.mongoClient = requireNonNull(mongoClient);
    +    }
    +
    +    @Override
    +    public void batchUpdate(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        Preconditions.checkState(instanceExists.exists(ryaInstanceName), "The instance: " + ryaInstanceName + " does not exist.");
    +
    +        verifyPCJState(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJResults(ryaInstanceName, pcjId, mongoClient);
    +        updatePCJMetadata(ryaInstanceName, pcjId, mongoClient);
    +    }
    +
    +    private void verifyPCJState(final String ryaInstanceName, final String pcjId, final MongoClient client) throws RyaClientException {
    +        try {
    +            // Fetch the Rya instance's details.
    +            final RyaDetailsRepository detailsRepo = new MongoRyaInstanceDetailsRepository(client, ryaInstanceName);
    +            final RyaDetails ryaDetails = detailsRepo.getRyaInstanceDetails();
    +
    +            // Ensure PCJs are enabled.
    +            if(!ryaDetails.getPCJIndexDetails().isEnabled()) {
    +                throw new RyaClientException("PCJs are not enabled for the Rya instance named '" + ryaInstanceName + "'.");
    +            }
    +
    +            // Ensure the PCJ exists.
    +            if(!ryaDetails.getPCJIndexDetails().getPCJDetails().containsKey(pcjId)) {
    +                throw new PCJDoesNotExistException("The PCJ with id '" + pcjId + "' does not exist within Rya instance '" + ryaInstanceName + "'.");
    +            }
    +
    +        } catch(final NotInitializedException e) {
    +            throw new InstanceDoesNotExistException("No RyaDetails are initialized for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        } catch (final RyaDetailsRepositoryException e) {
    +            throw new RyaClientException("Could not fetch the RyaDetails for the Rya instance named '" + ryaInstanceName + "'.", e);
    +        }
    +    }
    +
    +    private void updatePCJResults(final String ryaInstanceName, final String pcjId, final MongoClient client) throws InstanceDoesNotExistException, PCJDoesNotExistException, RyaClientException {
    +        // Things that have to be closed before we exit.
    +        Sail sail = null;
    +        SailConnection sailConn = null;
    +
    +        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(client, ryaInstanceName)) {
    +            // Create an instance of Sail backed by the Rya instance.
    +            sail = connectToRya(ryaInstanceName);
    +            final SailRepository sailRepo = new SailRepository(sail);
    +            final SailRepositoryConnection sailRepoConn = sailRepo.getConnection();
    +            // Purge the old results from the PCJ.
    +            try {
    +                pcjStorage.purge(pcjId);
    +            } catch (final PCJStorageException e) {
    +                throw new RyaClientException("Could not batch update PCJ with ID '" + pcjId + "' because the old " +
    +                        "results could not be purged from it.", e);
    +            }
    +
    +            // Parse the PCJ's SPARQL query.
    +            final PcjMetadata metadata = pcjStorage.getPcjMetadata(pcjId);
    +            final String sparql = metadata.getSparql();
    +            sailConn = sail.getConnection();
    +            final TupleQuery tupleQuery = sailRepoConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +
    +            // Execute the query.
    +            final List<VisibilityBindingSet> batch = new ArrayList<>(1000);
    +            tupleQuery.evaluate(new TupleQueryResultHandlerBase() {
    +                @Override
    +                public void handleSolution(final BindingSet bindingSet) throws TupleQueryResultHandlerException {
    +                    final VisibilityBindingSet result;
    +                    if(bindingSet instanceof VisibilityBindingSet) {
    --- End diff --
    
    This isn't necessary as TupleQuery will only return BindingSets.  


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127529895
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     *
    +     * @return Iterator of Lists (combos) of PCJs used to build an optimal query plan
    +     */
    +    @Override
    +    public Iterator<List<ExternalTupleSet>> getExternalSetCombos(final QuerySegment<ExternalTupleSet> segment) {
    +        final ValidIndexCombinationGenerator comboGen = new ValidIndexCombinationGenerator(segment.getOrderedNodes());
    +        return comboGen.getValidIndexCombos(getExternalSets(segment));
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     * @return List of PCJs for matching
    +     */
    +    @Override
    +    public List<ExternalTupleSet> getExternalSets(final QuerySegment<ExternalTupleSet> segment) {
    +        try {
    +            if(!init) {
    +                indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +                init = true;
    +            }
    +            final TupleExpr query = segment.getQuery().getTupleExpr();
    +            final IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(query, indexCache);
    +            final List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
    +            final List<ExternalTupleSet> tuples = new ArrayList<>();
    +            for (final ExternalTupleSet tuple: pcjs) {
    +                final QuerySegment<ExternalTupleSet> pcj = converter.setToSegment(tuple);
    +                if (segment.containsQuerySegment(pcj)) {
    +                    tuples.add(tuple);
    +                }
    +            }
    +            return tuples;
    +
    +        } catch (final Exception e) {
    +            throw new RuntimeException(e);
    --- End diff --
    
    throw something better


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160760999
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    --- End diff --
    
    done


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    The test that checks the output of the RyaDetailsFormatter needs to be updated.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

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


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160569220
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AccumuloIndexSetProvider.java ---
    @@ -0,0 +1,139 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.accumulo.ConfigUtils;
    +import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for Accumulo.
    + * This provider uses either user specified Accumulo configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If Accumulo configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public class AccumuloIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private static final Logger log = Logger.getLogger(AccumuloIndexSetProvider.class);
    +
    +    public AccumuloIndexSetProvider(final Configuration conf) {
    +        super(conf);
    +    }
    +
    +    public AccumuloIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final String tablePrefix = requireNonNull(conf.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX));
    +            final Connector conn = requireNonNull(ConfigUtils.getConnector(conf));
    +            List<String> tables = null;
    +
    +            if (conf instanceof RdfCloudTripleStoreConfiguration) {
    +                tables = ((RdfCloudTripleStoreConfiguration) conf).getPcjTables();
    +            }
    +            // this maps associates pcj table name with pcj sparql query
    +            final Map<String, String> indexTables = Maps.newLinkedHashMap();
    +
    +            try(final PrecomputedJoinStorage storage = new AccumuloPcjStorage(conn, tablePrefix)) {
    +                final PcjTableNameFactory pcjFactory = new PcjTableNameFactory();
    +
    +                final boolean tablesProvided = tables != null && !tables.isEmpty();
    +
    +                if (tablesProvided) {
    --- End diff --
    
    I'm not touching accumulo code.  this is just a repackage


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160531032
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java ---
    @@ -154,6 +154,5 @@ public StatefulMongoDBRdfConfiguration getConf() {
     
         @Override
         public void close() throws IOException {
    -//        if (mongoClient != null){ mongoClient.close(); }
    --- End diff --
    
    the red is showing it is getting deleted....


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160550164
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/InstallPrompt.java ---
    @@ -224,6 +224,10 @@ private InstallConfiguration promptMongoConfig(final String instanceName) throws
                 final boolean useTemporalIndexing = promptBoolean(prompt, Optional.of(true));
                 builder.setEnableTemporalIndex( useTemporalIndexing );
     
    +            prompt = makeFieldPrompt("Use PCJ Indexing", true);
    --- End diff --
    
    Since you added entity stuff back in, also add it here?


---

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/240/<h2>Build result: FAILURE</span></h2>[...truncated 32.39 KB...][INFO] Apache Rya Accumulo Pig ............................ SKIPPED[INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 01:59 min[INFO] Finished at: 2017-06-20T17:46:17+00:00[INFO] Final Memory: 46M/746M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.api: Too many files with unapproved license: 1 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator
 -rya-master-with-optionals-pull-requests/common/rya.api/target/rat.txt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.apichannel stoppedSetting status of 772ce96d774f29dacf291963a8de9cc915d22442 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/240/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160532306
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AccumuloIndexSetProvider.java ---
    @@ -0,0 +1,139 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.accumulo.ConfigUtils;
    +import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for Accumulo.
    + * This provider uses either user specified Accumulo configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If Accumulo configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public class AccumuloIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private static final Logger log = Logger.getLogger(AccumuloIndexSetProvider.class);
    +
    +    public AccumuloIndexSetProvider(final Configuration conf) {
    +        super(conf);
    +    }
    +
    +    public AccumuloIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final String tablePrefix = requireNonNull(conf.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX));
    +            final Connector conn = requireNonNull(ConfigUtils.getConnector(conf));
    +            List<String> tables = null;
    +
    +            if (conf instanceof RdfCloudTripleStoreConfiguration) {
    +                tables = ((RdfCloudTripleStoreConfiguration) conf).getPcjTables();
    +            }
    +            // this maps associates pcj table name with pcj sparql query
    +            final Map<String, String> indexTables = Maps.newLinkedHashMap();
    +
    +            try(final PrecomputedJoinStorage storage = new AccumuloPcjStorage(conn, tablePrefix)) {
    +                final PcjTableNameFactory pcjFactory = new PcjTableNameFactory();
    +
    +                final boolean tablesProvided = tables != null && !tables.isEmpty();
    +
    +                if (tablesProvided) {
    +                    // if tables provided, associate table name with sparql
    +                    for (final String table : tables) {
    +                        indexTables.put(table, storage.getPcjMetadata(pcjFactory.getPcjId(table)).getSparql());
    +                    }
    +                } else if (hasRyaDetails(tablePrefix, conn)) {
    +                    // If this is a newer install of Rya, and it has PCJ Details, then
    +                    // use those.
    +                    final List<String> ids = storage.listPcjs();
    +                    for (final String id : ids) {
    +                        indexTables.put(pcjFactory.makeTableName(tablePrefix, id), storage.getPcjMetadata(id).getSparql());
    +                    }
    +                } else {
    +                    // Otherwise figure it out by scanning tables.
    +                    final PcjTables pcjTables = new PcjTables();
    +                    for (final String table : conn.tableOperations().list()) {
    +                        if (table.startsWith(tablePrefix + "INDEX")) {
    +                            indexTables.put(table, pcjTables.getPcjMetadata(conn, table).getSparql());
    +                        }
    +                    }
    +                }
    +            }
    +
    +            // use table name sparql map (indexTables) to create {@link
    +            // AccumuloIndexSet}
    +            final List<ExternalTupleSet> index = Lists.newArrayList();
    +            if (indexTables.isEmpty()) {
    +                log.info("No Index found");
    +            } else {
    +                for (final String table : indexTables.keySet()) {
    +                    final String indexSparqlString = indexTables.get(table);
    +                    index.add(new AccumuloIndexSet(indexSparqlString, conf, table));
    --- End diff --
    
    Oh, there's Accumulo stuff listed here too.


---

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/249/<h2>Build result: FAILURE</span></h2>[...truncated 31.34 KB...][INFO] Apache Rya Spark Support ........................... SKIPPED[INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 29.979 s[INFO] Finished at: 2017-07-05T18:53:32+00:00[INFO] Final Memory: 44M/647M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.api: Too many files with unapproved license: 1 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/common/rya.api/target/rat.t
 xt -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.apichannel stoppedSetting status of 136372b715936076ca33fcc56cc4847efda2db24 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/249/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159754029
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java ---
    @@ -40,8 +40,8 @@
      */
     public class RyaToRdfConversions {
     
    -    public static URI convertURI(RyaURI uri) {
    --- End diff --
    
    RyaType is too general of an argument here.  Per our discussion, create a private method that does what you want.


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761790
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     * @param pcjName - The PCJ to add the results to.
    +     * @param results - The binding set results.
    +     */
    +    public void addResults(final String pcjName, final Collection<VisibilityBindingSet> results) {
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_NAME, pcjName);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(AUTHS_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ document with the provided name.
    +     * @param pcjName - The name of the PCJ document to purge. (not null)
    +     */
    +    public void purgePcjs(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // remove every doc for the pcj, except the metadata
    +        final Bson filter = new Document(PCJ_NAME, pcjName);
    +        pcjCollection.deleteMany(filter);
    +
    +        // reset cardinality
    +        final Bson query = new Document(PCJ_ID, getMetadataID(pcjName));
    +        final Bson update = new Document("$set", new Document(CARDINALITY_FIELD, 0));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Scan Rya for results that solve the PCJ's query and store them in the PCJ document.
    +     * <p>
    +     * This method assumes the PCJ document has already been created.
    +     *
    +     * @param pcjTableName - The name of the PCJ table that will receive the results. (not null)
    +     * @param ryaConn - A connection to the Rya store that will be queried to find results. (not null)
    +     * @throws PCJStorageException If results could not be written to the PCJ table,
    +     *   the PCJ table does not exist, or the query that is being execute
    +     *   was malformed.
    +     */
    +    public void populatePcj(final String pcjTableName, final RepositoryConnection ryaConn) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(ryaConn);
    +
    +        try {
    +            // Fetch the query that needs to be executed from the PCJ table.
    +            final PcjMetadata pcjMetadata = getPcjMetadata(pcjTableName);
    +            final String sparql = pcjMetadata.getSparql();
    +
    +            // Query Rya for results to the SPARQL query.
    +            final TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL, sparql);
    +            final TupleQueryResult results = query.evaluate();
    +
    +            // Load batches of 1000 of them at a time into the PCJ table
    +            final Set<VisibilityBindingSet> batch = new HashSet<>(1000);
    +            while(results.hasNext()) {
    +                final VisibilityBindingSet bs = new VisibilityBindingSet(results.next());
    +                batch.add( bs );
    +                System.out.println(bs.toString());
    +                if(batch.size() == 1000) {
    +                    addResults(pcjTableName, batch);
    +                    batch.clear();
    +                }
    +            }
    +
    +            if(!batch.isEmpty()) {
    +                addResults(pcjTableName, batch);
    +            }
    +
    +        } catch (RepositoryException | MalformedQueryException | QueryEvaluationException e) {
    +            throw new PCJStorageException("Could not populate a PCJ document with Rya results for the pcj named: " + pcjTableName, e);
    +        }
    +    }
    +
    +    /**
    +     * List the document names of the PCJ index tables that are stored in MongoDB
    +     * for this instance of Rya.
    +     *
    +     * @return A list of pcj document names that hold PCJ index data for the current
    +     *   instance of Rya.
    +     */
    +    public List<String> listPcjDocuments() {
    +        final List<String> pcjNames = new ArrayList<>();
    +
    +        //This Bson string reads as:
    +        //{} - no search criteria: find all
    +        //{ _id: 1 } - only return the _id, which is the PCJ name.
    +        final FindIterable<Document> rez = pcjCollection.find((Bson) JSON.parse("{ }, { " + PCJ_ID + ": 1 , _id: 0}"));
    +        final Iterator<Document> iter = rez.iterator();
    +        while(iter.hasNext()) {
    +            pcjNames.add(iter.next().get(PCJ_ID).toString().replace("_METADATA", ""));
    +        }
    +
    +        return pcjNames;
    +    }
    +
    +    /**
    +     * Returns all of the results of a PCJ.
    +     *
    +     * @param pcjName - The PCJ to get the results for. (not null)
    +     * @return The authorized PCJ results.
    +     */
    +    public CloseableIterator<BindingSet> listResults(final String pcjName) {
    +        requireNonNull(pcjName);
    +
    +        // get all results based on pcjName
    +        return queryForBindings(new Document(PCJ_NAME, pcjName));
    +    }
    +
    +    /**
    +     * Retrieves the stored {@link BindingSet} results for the provided pcjName.
    +     *
    +     * @param pcjName - The pcj to retrieve results for.
    +     * @param authorizations - The authorizations of the user to restrict results.
    +     * @param bindingset - The collection of {@link BindingSet}s to restrict results.
    --- End diff --
    
    restrictionBindings


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160798118
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    Oh that's a good point. The PCJ ID actually would uniquely identify the metadata.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160541859
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +
    +import com.mongodb.MongoClient;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private final MongoConnectionDetails connectionDetails;
    --- End diff --
    
    This field is not used.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160831020
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    --- End diff --
    
    PCJ_METADATA_ID


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532218
  
    --- Diff: extras/rya.benchmark/src/main/java/org/apache/rya/benchmark/query/PCJOptimizerBenchmark.java ---
    @@ -317,8 +318,8 @@ private static String buildChainedSPARQL(final List<String> vars) {
     
             // Build the SPARQL query from the pieces.
             return "select " + Joiner.on(" ").join(vars) + " where { " +
    -                    Joiner.on(" . ").join(statementPatterns) +
    -                " . }" ;
    +        Joiner.on(" . ").join(statementPatterns) +
    --- End diff --
    
    nit fix whitespace


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132187057
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    --- End diff --
    
    I think that you should make this name more Mongo centric.  PcjQueryNode sounds very general purpose and makes it seem like this class is DB agnostic.  We should have similar naming conventions for the two PCJ nodes.  Currently the other node is AccumuloIndexSet.  If you don't like MongoIndexSet, we can rename that to AccumuloPcjNode and rename this class to MongoPcjNode.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160792306
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,451 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    --- End diff --
    
    This has an _id field, right? Does it use a random mongo ID right now?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160833781
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r161082144
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,446 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_ID]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_ID],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_METADATA_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_ID = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document makeMetadataDocument(final String pcjId, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_METADATA_ID, makeMetadataID(pcjId))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_FIELD, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata. The initial pcj results
    +     * will be empty.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjId, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(makeMetadataDocument(pcjId, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ documents will be left in Mongo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the PCJ results document. (not null)
    +     * @throws PCJStorageException The PCJ documents could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjId,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(ryaConn);
    +        checkNotNull(pcjId);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjId, sparql);
    +
    +        // Load historic matches from Rya into the PCJ results document.
    +        populatePcj(pcjId, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ Id.
    +     *
    +     * @param pcjId - The Id of the PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ metadata document does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjId) throws PCJStorageException {
    +        requireNonNull(pcjId);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_METADATA_ID, makeMetadataID(pcjId))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjId + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_FIELD);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +
    +        return new PcjMetadata(sparql, cardinality, varOrder);
    +    }
    +
    +    /**
    +     * Adds binding set results to a specific PCJ.
    +     *
    +     * @param pcjId - Uniquely identifies a PCJ within Rya. (not null)
    +     * @param results - The binding set results. (not null)
    +     */
    +    public void addResults(final String pcjId, final Collection<VisibilityBindingSet> results) {
    +        checkNotNull(pcjId);
    +        checkNotNull(results);
    +
    +        final List<Document> pcjDocs = new ArrayList<>();
    +        for (final VisibilityBindingSet vbs : results) {
    +            // each binding gets it's own doc.
    +            final Document bindingDoc = new Document(PCJ_ID, pcjId);
    +            vbs.forEach(binding -> {
    +                final RyaType type = RdfToRyaConversions.convertValue(binding.getValue());
    +                bindingDoc.append(binding.getName(),
    +                        new Document()
    +                        .append(BINDING_TYPE, type.getDataType().stringValue())
    +                        .append(BINDING_VALUE, type.getData())
    +                        );
    +            });
    +            bindingDoc.append(VISIBILITIES_FIELD, vbs.getVisibility());
    +            pcjDocs.add(bindingDoc);
    +        }
    +        pcjCollection.insertMany(pcjDocs);
    +
    +        // update cardinality in the metadata doc.
    +        final int appendCardinality = pcjDocs.size();
    +        final Bson query = new Document(PCJ_METADATA_ID, makeMetadataID(pcjId));
    +        final Bson update = new Document("$inc", new Document(CARDINALITY_FIELD, appendCardinality));
    +        pcjCollection.updateOne(query, update);
    +    }
    +
    +    /**
    +     * Purges all results from the PCJ results document with the provided name.
    --- End diff --
    
    with the provided [id].


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160533521
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoCreatePCJ.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +import java.util.Set;
    +
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoCreatePCJ implements CreatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoCreatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoCreatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoCreatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public String createPCJ(final String ryaInstanceName, final String sparql, final Set<ExportStrategy> strategies) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(sparql);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    +        final MongoCredential cred;
    +        final MongoClient client;
    +
    +        final ServerAddress addr = new ServerAddress(connectionDetails.getHostname(), connectionDetails.getPort());
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160454168
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,172 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private static final Logger log = Logger.getLogger(PcjQueryNode.class);
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     * Creates a new {@link PcjQueryNode}.
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     *
    +     * @throws MalformedQueryException - The SPARQL query needs to contain a projection.
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs) throws MalformedQueryException {
    --- End diff --
    
    @params don't match


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160514509
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoDeletePCJ.java ---
    @@ -0,0 +1,96 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +
    +import org.apache.rya.api.client.DeletePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link DeletePCJ}.
    + */
    +public class MongoDeletePCJ implements DeletePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoDeletePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoDeletePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoDeletePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public void deletePCJ(final String ryaInstanceName, final String pcjId) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(pcjId);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    +        final MongoCredential cred;
    +        final MongoClient client;
    +
    +        final ServerAddress addr = new ServerAddress(connectionDetails.getHostname(), connectionDetails.getPort());
    +        if(username.isPresent()) {
    +            cred = MongoCredential.createCredential(username.get(), ryaInstanceName, connectionDetails.getPassword().get());
    +            client = new MongoClient(addr, Lists.newArrayList(cred));
    +        } else {
    +            client = new MongoClient(addr);
    +        }
    +
    +
    +        //pcjStore doesn't need to be closed since it doesn't need to release any resources.
    +        @SuppressWarnings("resource")
    +        final MongoPcjStorage pcjStore = new MongoPcjStorage(client, ryaInstanceName);
    --- End diff --
    
    Same comment about not suppressing warnings.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132175692
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,125 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private final MongoClient client;
    +    private final MongoDBRdfConfiguration mongoConf;
    +
    +    public MongoPcjIndexSetProvider(final Configuration conf, final MongoClient client) {
    +        super(conf);
    +        this.client = client;
    --- End diff --
    
    Preconditions


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160535354
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    --- End diff --
    
    a lot of the tests would need to be repackaged to allow that


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160550034
  
    --- Diff: extras/shell/src/main/java/org/apache/rya/shell/util/InstallPrompt.java ---
    @@ -91,14 +91,14 @@ public InstallConfiguration promptInstallConfiguration(final String instanceName
                 checkState(storageType.isPresent(), "The shell must be connected to a storage to use the install prompt.");
     
                 switch(sharedShellState.getShellState().getStorageType().get()) {
    -                case ACCUMULO:
    -                    return promptAccumuloConfig(instanceName);
    +            case ACCUMULO:
    +                return promptAccumuloConfig(instanceName);
     
    -                case MONGO:
    -                    return promptMongoConfig(instanceName);
    +            case MONGO:
    +                return promptMongoConfig(instanceName);
     
    -                default:
    -                    throw new IllegalStateException("Unsupported storage type: " + storageType.get());
    +            default:
    +                throw new IllegalStateException("Unsupported storage type: " + storageType.get());
    --- End diff --
    
    Messsed up switch white space.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160810833
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    I could split it up into pcj_metadata collection and pcj_results collcetion


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160575498
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
    +            final MongoClient client = mongoConf.getMongoClient();
    +            final MongoPcjDocuments pcjDocs = new MongoPcjDocuments(client, mongoConf.getRyaInstanceName());
    +            List<String> tables = null;
    --- End diff --
    
    The mongo side is significantly different from the accumulo side when it comes to PCJ. A bunch of the stuff that was in there has to do with legacy compatibility.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761257
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    --- End diff --
    
    artifact from when I thought I didn't need var orders, removing it


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r132280543
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,418 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.VisibilityBindingSet;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    --- End diff --
    
    n is just showing that its the nth var.  the dots are showing any number between


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

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/253/<h2>Build result: ABORTED</span></h2>[...truncated 6.03 MB...][INFO] [INFO] --- apache-rat-plugin:0.11:check (check-licenses) @ rya.console ---[INFO] 51 implicit excludes (use -debug for more details).[INFO] Exclude: spring-shell.log[INFO] 25 resources included (use -debug for more details)[INFO] Rat check: Summary of files. Unapproved: 0 unknown: 0 generated: 0 approved: 24 licence.[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 27 iota=54[INFO] [INFO] --- maven-enforcer-plugin:1.3.1:enforce (enforce-mvn) @ rya.console ---[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 27 iota=54[INFO] [INFO] --- jacoco-maven-plugin:0.7.6.201602180812:prepare-agent (default-prepare-agent) @ rya.console ---[INFO] argLine set to -javaagent:/home/
 jenkins/jenkins-slave/maven-repositories/0/org/jacoco/org.jacoco.agent/0.7.6.201602180812/org.jacoco.agent-0.7.6.201602180812-runtime.jar=destfile=/home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.console/target/jacoco.exec[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 27 iota=54[INFO] [INFO] --- license-maven-plugin:2.6:format (default) @ rya.console ---[INFO] Updating license headers...[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 27 iota=54[INFO] [INFO] --- maven-remote-resources-plugin:1.5:process (default) @ rya.console ---Build was abortedchannel stoppedSetting status of 53fdf96dab2b48adc9429d89d97c84d4c6c57b3b to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/253/ and message: 'Build finished. 'Using contex
 t: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159934928
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    --- End diff --
    
    Did this ever get renamed to something more Mongo specific?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160517997
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    --- End diff --
    
    nit: Could be this.conf = requireNonNull(conf);


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160539494
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    --- End diff --
    
    I think this hunk of doc is missing the variable orders field.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528422
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    +    private static final Logger log = Logger.getLogger(MongoPrecomputedJoinIndexer.class);
    +    private final AtomicReference<MongoClient> client = new AtomicReference<>();
    +
    +    /**
    +     * Creates a new {@link MongoPrecomputedJoinIndexer}.
    +     * <p>
    +     * Note: This constructor should only be used when the {@link MongoClient}
    +     * is provided by the configuration when calling
    +     * {@link #getPcjStorage(Configuration)}.
    +     */
    +    public MongoPrecomputedJoinIndexer() {
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPrecomputedJoinIndexer} with a provided
    +     * {@link MongoClient}.
    +     *
    +     * @param client
    --- End diff --
    
    nit: fix


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160761195
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127528847
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        this.tablename = tablename;
    +        final SPARQLParser sp = new SPARQLParser();
    +        final ParsedTupleQuery pq = (ParsedTupleQuery) sp.parseQuery(sparql, null);
    +        final TupleExpr te = pq.getTupleExpr();
    +        Preconditions.checkArgument(PCJOptimizerUtilities.isPCJValid(te), "TupleExpr is an invalid PCJ.");
    +
    +        final Optional<Projection> projection = new ParsedQueryUtil().findProjection(pq);
    +        if (!projection.isPresent()) {
    +            throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
    +        }
    +        setProjectionExpr(projection.get());
    +    }
    +
    +    /**
    +     *
    +     * @param accCon
    +     *            - connection to a valid Accumulo instance
    +     * @param tablename
    +     *            - name of an existing PCJ table
    +     * @throws PCJStorageException
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     */
    +    public PcjQueryNode(final Configuration conf, final String tablename)
    +            throws PCJStorageException, MalformedQueryException {
    +        indexer = new MongoPrecomputedJoinIndexer();
    +        pcjDocs = indexer.getPcjStorage(conf);
    +        this.tablename = tablename;
    +    }
    +
    +    /**
    +     * returns size of table for query planning
    +     */
    +    @Override
    +    public double cardinality() {
    +        double cardinality = 0;
    +        try {
    +            cardinality = pcjDocs.getPcjMetadata(tablename).getCardinality();
    +        } catch (final PcjException e) {
    +            e.printStackTrace();
    --- End diff --
    
    don't print stack trace


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

[GitHub] incubator-rya issue #172: [WIP] RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172
  
    
    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/252/<h2>Build result: FAILURE</span></h2>[...truncated 4.45 MB...][INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 07:28 min[INFO] Finished at: 2017-07-06T19:51:16+00:00[INFO] Final Memory: 98M/976M[INFO] ------------------------------------------------------------------------Waiting for Jenkins to finish collecting data[ERROR] Failed to execute goal org.apache.rat:apache-rat-plugin:0.11:check (check-licenses) on project rya.indexing: Too many files with unapproved license: 4 See RAT report in: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/indexing/target/rat.txt -> [Help 1][E
 RROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.indexingchannel stoppedSetting status of 0739172459b0c14239a15dbec1a28e406f4d14b9 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/252/ and message: 'Build finished. 'Using context: Jenkins: clean package -Pgeoindexing



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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r127532069
  
    --- Diff: extras/rya.benchmark/src/main/gen/META-INF/sun-jaxb.episode ---
    @@ -5,7 +5,7 @@
     This file was generated by the JavaTM Architecture for XML Binding(JAXB) Reference Implementation, v2.2.11 
     See <a href="http://java.sun.com/xml/jaxb">http://java.sun.com/xml/jaxb</a> 
     Any modifications to this file will be lost upon recompilation of the source schema. 
    -Generated on: 2016.12.16 at 01:22:14 PM PST 
    +Generated on: 2017.07.06 at 03:13:11 PM EDT 
    --- End diff --
    
    what....


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r139545490
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    --- End diff --
    
    Since the updater is fetched through the PcjUpdaterSupplierFactory, a lot of the functionality is already abstracted out.  This BasePcjIndexer should be in the Accumulo/Fluo hierarchy, and the PcjUpdaterSupplierFactory returns the fluo updater, and the only thing that would need doing on the mongo side is adding the updater to the factory.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160513812
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/api/client/mongo/MongoCreatePCJ.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.api.client.mongo;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.Optional;
    +import java.util.Set;
    +
    +import org.apache.rya.api.client.CreatePCJ;
    +import org.apache.rya.api.client.InstanceDoesNotExistException;
    +import org.apache.rya.api.client.InstanceExists;
    +import org.apache.rya.api.client.RyaClientException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +import com.mongodb.MongoClient;
    +import com.mongodb.MongoCredential;
    +import com.mongodb.ServerAddress;
    +
    +/**
    + * A Mongo implementation of {@link CreatePCJ}.
    + */
    +public class MongoCreatePCJ implements CreatePCJ {
    +    private static final Logger log = LoggerFactory.getLogger(MongoCreatePCJ.class);
    +
    +    private final MongoConnectionDetails connectionDetails;
    +    private final InstanceExists instanceExists;
    +
    +    /**
    +     * Constructs an instance of {@link MongoCreatePCJ}.
    +     *
    +     * @param connectionDetails - Details to connect to the server. (not null)
    +     * @param instanceExists - The interactor used to check if a Rya instance exists. (not null)
    +     */
    +    public MongoCreatePCJ(
    +            final MongoConnectionDetails connectionDetails,
    +            final MongoInstanceExists instanceExists) {
    +        this.connectionDetails = requireNonNull(connectionDetails);
    +        this.instanceExists = requireNonNull(instanceExists);
    +    }
    +
    +    @Override
    +    public String createPCJ(final String ryaInstanceName, final String sparql, final Set<ExportStrategy> strategies) throws InstanceDoesNotExistException, RyaClientException {
    +        requireNonNull(ryaInstanceName);
    +        requireNonNull(sparql);
    +
    +        // Ensure the Rya Instance exists.
    +        if (!instanceExists.exists(ryaInstanceName)) {
    +            throw new InstanceDoesNotExistException(String.format("There is no Rya instance named '%s'.", ryaInstanceName));
    +        }
    +
    +        final Optional<String> username = connectionDetails.getUsername();
    +        final MongoCredential cred;
    +        final MongoClient client;
    +
    +        final ServerAddress addr = new ServerAddress(connectionDetails.getHostname(), connectionDetails.getPort());
    --- End diff --
    
    The mongo client this uses should be provided to the interactor instead of having it create a new one. If creating the client is the only thing the connection details are being used for, then you could remove them from the constructor and replace them with a MongoClient.
      


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159803009
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPrecomputedJoinIndexer.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +
    +import com.mongodb.MongoClient;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class MongoPrecomputedJoinIndexer extends BasePcjIndexer {
    --- End diff --
    
    Yeah, still not getting what you're doing here.  The base class appears to be dealing with entities and there is no updater for the base class to interact with.  In general, you wouldn't want the base class to interact with the updater class given that any class that extends it would be locked into using that updater.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160546330
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    --- End diff --
    
    This parameter does not exist.
      


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160452112
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/instance/RyaDetailsToConfiguration.java ---
    @@ -51,17 +51,29 @@ public static void addRyaDetailsToConfiguration(final RyaDetails details, final
     
             checkAndSet(conf, ConfigurationFields.USE_ENTITY, details.getEntityCentricIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_FREETEXT, details.getFreeTextIndexDetails().isEnabled());
    -      //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
    +        //RYA-215        checkAndSet(conf, ConfigurationFields.USE_GEO, details.getGeoIndexDetails().isEnabled());
             checkAndSet(conf, ConfigurationFields.USE_TEMPORAL, details.getTemporalIndexDetails().isEnabled());
             final PCJIndexDetails pcjDetails = details.getPCJIndexDetails();
    -        if (pcjDetails.isEnabled() && pcjDetails.getFluoDetails().isPresent()) {
    -            checkAndSet(conf, ConfigurationFields.USE_PCJ_UPDATER, true);
    -            conf.set(ConfigurationFields.FLUO_APP_NAME, pcjDetails.getFluoDetails().get().getUpdateAppName());
    -            conf.set(ConfigurationFields.PCJ_UPDATER_TYPE, "FLUO");
    -            conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "ACCUMULO");
    +        
    +        if(conf.getBoolean(ConfigurationFields.USE_MONGO, false) ) {
    +        	if(pcjDetails.isEnabled()) {
    +        		conf.set(ConfigurationFields.PCJ_STORAGE_TYPE, "MONGO");
    --- End diff --
    
    Can "MONGO" be replaced with PrecomputedJoinStorageType.MONGO.toString()?  Or is the dependency for that class not in rya.api?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r160796664
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    --- End diff --
    
    Having dealt with multiple versions of pcj id on the Fluo side, I think its probably best to remove the ambiguity and delete the "metadata" suffix.  Just use pcjName (which I assume is a uuid) for the PCJ_ID field.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160760932
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    --- End diff --
    
    removed doc


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r159924600
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java ---
    @@ -40,8 +40,8 @@
      */
     public class RyaToRdfConversions {
     
    -    public static URI convertURI(RyaURI uri) {
    --- End diff --
    
    done


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160547226
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "uri";
    +    private static final String AUTHS_FIELD = "auths";
    +    private static final String PCJ_NAME = "pcjName";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String getMetadataID(final String pcjName) {
    +        return pcjName + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * @param pcjName - The name of the PCJ. (not null)
    +     * @param sparql - The sparql query the PCJ will use.
    +     * @return The document built around the provided metadata.
    +     * @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public Document getMetadataDocument(final String pcjName, final String sparql) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +        requireNonNull(sparql);
    +
    +        final Set<VariableOrder> varOrders;
    +        try {
    +            varOrders = pcjVarOrderFactory.makeVarOrders(sparql);
    +        } catch (final MalformedQueryException e) {
    +            throw new PCJStorageException("Can not create the PCJ. The SPARQL is malformed.", e);
    +        }
    +
    +        return new Document()
    +                .append(PCJ_ID, getMetadataID(pcjName))
    +                .append(SPARQL_FIELD, sparql)
    +                .append(CARDINALITY_FIELD, 0)
    +                .append(VAR_ORDER_ID, varOrders);
    +
    +    }
    +
    +    /**
    +     * Creates a new PCJ based on the provided metadata.  The initial pcj results will be empty.
    +     * @param pcjName - The unique name of the PCJ.
    +     * @param varOrders - The {@link VariableOrder}s.
    +     * @param sparql - The query the pcj is assigned to.
    +     * @throws @throws PCJStorageException - Thrown when the sparql query is malformed.
    +     */
    +    public void createPcj(final String pcjName, final String sparql) throws PCJStorageException {
    +        pcjCollection.insertOne(getMetadataDocument(pcjName, sparql));
    +    }
    +
    +    /**
    +     * Creates a new PCJ document and populates it by scanning an instance of
    +     * Rya for historic matches.
    +     * <p>
    +     * If any portion of this operation fails along the way, the partially
    +     * create PCJ table will be left in Accumulo.
    +     *
    +     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
    +     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
    +     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
    +     * @throws PCJStorageException The PCJ table could not be create or the
    +     *     values from Rya were not able to be loaded into it.
    +     */
    +    public void createAndPopulatePcj(
    +            final RepositoryConnection ryaConn,
    +            final String pcjTableName,
    +            final String sparql) throws PCJStorageException {
    +        checkNotNull(pcjTableName);
    +        checkNotNull(sparql);
    +
    +        // Create the PCJ document in Mongo.
    +        createPcj(pcjTableName, sparql);
    +
    +        // Load historic matches from Rya into the PCJ table.
    +        populatePcj(pcjTableName, ryaConn);
    +    }
    +
    +    /**
    +     * Gets the {@link PcjMetadata} from a provided PCJ name.
    +     *
    +     * @param pcjName - The PCJ to get from MongoDB. (not null)
    +     * @return - The {@link PcjMetadata} of the Pcj specified.
    +     * @throws PCJStorageException The PCJ Table does not exist.
    +     */
    +    public PcjMetadata getPcjMetadata(final String pcjName) throws PCJStorageException {
    +        requireNonNull(pcjName);
    +
    +        // since query by ID, there will only be one.
    +        final Document result = pcjCollection.find(new Document(PCJ_ID, getMetadataID(pcjName))).first();
    +
    +        if(result == null) {
    +            throw new PCJStorageException("The PCJ: " + pcjName + " does not exist.");
    +        }
    +
    +        final String sparql = result.getString(SPARQL_FIELD);
    +        final int cardinality = result.getInteger(CARDINALITY_FIELD, 0);
    +        final List<List<String>> varOrders= (List<List<String>>) result.get(VAR_ORDER_ID);
    +        final Set<VariableOrder> varOrder = new HashSet<>();
    +        for(final List<String> vars : varOrders) {
    +            varOrder.add(new VariableOrder(vars));
    +        }
    +        //MongoDB does not need to use VarOrders
    --- End diff --
    
    Why is this comment here? What exactly are the variable orders being used for?


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132182885
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    --- End diff --
    
    I'm not sure what this class is interacting with.  The basic components of our PCJ framework are the matcher framework for query optimization, the storage layer, and the indexer layer.  It seems like this is related to the indexer layer.  But the indexer layer is meant to interact with the updater (whatever observer framework we use to maintain the PCJs).  Given that there is currently no updater in place, what is the purpose of BasePcjIndexer, PcjIndexer, and MongoPrecomputedJoinIndexer?  I can understand including abstract classes and interfaces just to have them in place when an updater is incorporated, but some of these our concrete implementations.  So what are they interacting with?


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r159802759
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    --- End diff --
    
    Yeah, I don't get what this method or this class is doing.  In the logic below, it appears that you are dealing with entities, which is not correct.  The PCJ indexer feeds the updater statements.  You maintain in your response to my comments about the MongoPCJIndexer that this class does all of the "heavy lifting".  That seems impossible to me given that this base class appears to deal with entities and there is no updater to interact with.  


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r129946427
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    + * {@link RyaStatement}s are added to/removed from the Rya instance.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public abstract class BasePcjIndexer implements PcjIndexer, MongoSecondaryIndex {
    +
    +    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
    +    private final AtomicReference<MongoPcjDocuments> pcjDocs = new AtomicReference<>();
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        requireNonNull(conf);
    +        pcjDocs.set( getPcjStorage(conf) );
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return configuration.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        requireNonNull(statement);
    +        storeStatements( singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        requireNonNull(statements);
    +
    +        final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
    +            .collect(groupingBy(RyaStatement::getSubject));
    +
    +        for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
    +            try {
    +                updateEntity(entry.getKey(), entry.getValue());
    +            } catch (final EntityStorageException e) {
    +                throw new IOException("Failed to update the Entity index.", e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Updates a {@link Entity} to reflect new {@link RyaStatement}s.
    +     *
    +     * @param subject - The Subject of the {@link Entity} the statements are for. (not null)
    +     * @param statements - Statements that the {@link Entity} will be updated with. (not null)
    +     */
    +    private void updateEntity(final RyaURI subject, final Collection<RyaStatement> statements) throws EntityStorageException {
    +        requireNonNull(subject);
    --- End diff --
    
    May want to change the name of this method -- Entity doesn't make sense in a pcj context.  


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160515766
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/BasePcjIndexer.java ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkState;
    +import static java.util.Collections.singleton;
    +import static java.util.Objects.requireNonNull;
    +import static java.util.stream.Collectors.groupingBy;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaURI;
    +import org.apache.rya.indexing.entity.model.Entity;
    +import org.apache.rya.indexing.entity.storage.EntityStorage;
    +import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoSecondaryIndex;
    +import org.openrdf.model.URI;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * A base class that may be used to update an {@link EntityStorage} as new
    --- End diff --
    
    Entity Storage?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160570068
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    --- End diff --
    
    well.... square brackets denotes list/set so really the docs should not have them


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160530921
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java ---
    @@ -40,8 +40,8 @@
      */
     public class RyaToRdfConversions {
     
    -    public static URI convertURI(RyaURI uri) {
    -        return new URIImpl(uri.getData());
    +    public static URI convertURI(RyaType value) {
    --- End diff --
    
    yet another thing git lost to time.
    I feel weird documenting just one function I'm just going back to.  


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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

    https://github.com/apache/incubator-rya/pull/172#discussion_r132188383
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/PcjQueryNode.java ---
    @@ -0,0 +1,184 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.rya.api.utils.IteratorWrapper;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.external.tupleSet.ParsedQueryUtil;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.storage.PcjException;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.rdftriplestore.evaluation.ExternalBatchingIterator;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.algebra.Projection;
    +import org.openrdf.query.algebra.TupleExpr;
    +import org.openrdf.query.parser.ParsedTupleQuery;
    +import org.openrdf.query.parser.sparql.SPARQLParser;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +import info.aduna.iteration.CloseableIteration;
    +
    +/**
    + * Indexing Node for PCJs expressions to be inserted into execution plan to
    + * delegate entity portion of query to {@link MongoPrecomputedJoinIndexer}.
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class PcjQueryNode extends ExternalTupleSet implements ExternalBatchingIterator {
    +    private final String tablename;
    +    private final PcjIndexer indexer;
    +    private final MongoPcjDocuments pcjDocs;
    +
    +    /**
    +     *
    +     * @param sparql - name of sparql query whose results will be stored in PCJ table
    +     * @param conf - Rya Configuration
    +     * @param tablename - name of an existing PCJ table
    +     * @throws MalformedQueryException
    +     * @throws SailException
    +     * @throws QueryEvaluationException
    +     * @throws TableNotFoundException
    +     * @throws AccumuloSecurityException
    +     * @throws AccumuloException
    +     * @throws PCJStorageException
    +     */
    +    public PcjQueryNode(final String sparql, final String tablename, final MongoPcjDocuments pcjDocs)
    +            throws MalformedQueryException, SailException, QueryEvaluationException, TableNotFoundException,
    +            AccumuloException, AccumuloSecurityException, PCJStorageException {
    +        this.pcjDocs = checkNotNull(pcjDocs);
    +        indexer = new MongoPrecomputedJoinIndexer();
    --- End diff --
    
    So, the MongoPrecomputedJoinIndexer and the PcjQueryNode do not need to talk to each other.  Again, the PrecomputedJoinIndexer is for ingesting data into the Updater, while the PcjQueryNode is a placeholder for the sub query that the PCJ matches.


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

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160539928
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,438 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [table_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjName: [table_name],
    + *   auths: [auths]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: value
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_ID = "varOrders";
    --- End diff --
    
    Why call it in an ID instead of VAR_ORDERS_FIELD?


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160531931
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AccumuloIndexSetProvider.java ---
    @@ -0,0 +1,139 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
    +import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.accumulo.ConfigUtils;
    +import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.sail.SailException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for Accumulo.
    + * This provider uses either user specified Accumulo configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If Accumulo configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public class AccumuloIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    private static final Logger log = Logger.getLogger(AccumuloIndexSetProvider.class);
    +
    +    public AccumuloIndexSetProvider(final Configuration conf) {
    +        super(conf);
    +    }
    +
    +    public AccumuloIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        requireNonNull(conf);
    +        try {
    +            final String tablePrefix = requireNonNull(conf.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX));
    +            final Connector conn = requireNonNull(ConfigUtils.getConnector(conf));
    +            List<String> tables = null;
    +
    +            if (conf instanceof RdfCloudTripleStoreConfiguration) {
    +                tables = ((RdfCloudTripleStoreConfiguration) conf).getPcjTables();
    +            }
    +            // this maps associates pcj table name with pcj sparql query
    +            final Map<String, String> indexTables = Maps.newLinkedHashMap();
    +
    +            try(final PrecomputedJoinStorage storage = new AccumuloPcjStorage(conn, tablePrefix)) {
    +                final PcjTableNameFactory pcjFactory = new PcjTableNameFactory();
    +
    +                final boolean tablesProvided = tables != null && !tables.isEmpty();
    +
    +                if (tablesProvided) {
    --- End diff --
    
    This code is basically the exact same code as the Mongo one. I think th eonly unique part is which implementation of PrecomputedJoinStorage is being used.
      


---

[GitHub] incubator-rya issue #172: RYA-303 Mongo PCJ Support

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

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



---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160518561
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/provider/AbstractPcjIndexSetProvider.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.matching.provider;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.instance.RyaDetails;
    +import org.apache.rya.indexing.IndexPlanValidator.IndexedExecutionPlanGenerator;
    +import org.apache.rya.indexing.IndexPlanValidator.ValidIndexCombinationGenerator;
    +import org.apache.rya.indexing.external.matching.ExternalSetProvider;
    +import org.apache.rya.indexing.external.matching.QuerySegment;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.PCJOptimizerUtilities;
    +import org.apache.rya.indexing.pcj.matching.PCJToSegmentConverter;
    +import org.openrdf.query.algebra.TupleExpr;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Abstraction of {@link ExternalSetProvider} that provides {@link ExternalTupleSet}s.
    + * Implementations of this use either user specified configuration information or user a specified
    + * List of ExternalTupleSets to populate an internal cache of ExternalTupleSets.  If a configuration
    + * is provided, the provider connects to an instance of RyaDetails and populates the cache with
    + * PCJs registered in RyaDetails.
    + */
    +public abstract class AbstractPcjIndexSetProvider implements ExternalSetProvider<ExternalTupleSet> {
    +    protected static final Logger log = Logger.getLogger(AbstractPcjIndexSetProvider.class);
    +    protected static final PCJToSegmentConverter converter = new PCJToSegmentConverter();
    +    protected List<ExternalTupleSet> indexCache;
    +    protected final Configuration conf;
    +    protected boolean init = false;
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based on configuration only.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +    }
    +
    +    /**
    +     * Creates a new {@link AbstractPcjIndexSetProvider} based user provided {@link ExternalTupleSet}s.
    +     * @param conf - The {@link Configuration} used to connect to {@link RyaDetails}.
    +     * @param indices - The {@link ExternalTupleSet}s to populate the internal cache.
    +     */
    +    public AbstractPcjIndexSetProvider(final Configuration conf, final List<ExternalTupleSet> indices) {
    +        requireNonNull(conf);
    +        this.conf = conf;
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +
    +    /**
    +     *
    +     * @param indices
    +     */
    +    @VisibleForTesting
    +    public void setIndices(final List<ExternalTupleSet> indices) {
    +        indexCache = indices;
    +        init = true;
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     *
    +     * @return Iterator of Lists (combos) of PCJs used to build an optimal query plan
    +     */
    +    @Override
    +    public Iterator<List<ExternalTupleSet>> getExternalSetCombos(final QuerySegment<ExternalTupleSet> segment) {
    +        final ValidIndexCombinationGenerator comboGen = new ValidIndexCombinationGenerator(segment.getOrderedNodes());
    +        return comboGen.getValidIndexCombos(getExternalSets(segment));
    +    }
    +
    +    /**
    +     * @param segment - QuerySegment used to get relevant queries form index cache for matching
    +     * @return List of PCJs for matching
    +     */
    +    @Override
    +    public List<ExternalTupleSet> getExternalSets(final QuerySegment<ExternalTupleSet> segment) {
    +        try {
    +            if(!init) {
    +                indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +                init = true;
    +            }
    +            final TupleExpr query = segment.getQuery().getTupleExpr();
    +            final IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(query, indexCache);
    +            final List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
    +            final List<ExternalTupleSet> tuples = new ArrayList<>();
    +            for (final ExternalTupleSet tuple: pcjs) {
    +                final QuerySegment<ExternalTupleSet> pcj = converter.setToSegment(tuple);
    +                if (segment.containsQuerySegment(pcj)) {
    +                    tuples.add(tuple);
    +                }
    +            }
    +            return tuples;
    +
    +        } catch (final Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    /**
    +     * @return The size of the set index cache.
    +     * @throws Exception
    +     */
    +    public int size() throws Exception {
    +        if (!init) {
    +            indexCache = PCJOptimizerUtilities.getValidPCJs(getIndices());
    +            init = true;
    +        }
    +        return indexCache.size();
    +    }
    +
    +    /**
    +     * @param conf - client configuration
    --- End diff --
    
    This documentation is out of date.


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160813037
  
    --- Diff: extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/pcj/MongoPcjIndexSetProvider.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.mongodb.pcj;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.rya.api.instance.RyaDetailsRepository;
    +import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
    +import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
    +import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjDocuments;
    +import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
    +import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
    +import org.openrdf.query.MalformedQueryException;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.mongodb.MongoClient;
    +
    +/**
    + * Implementation of {@link AbstractPcjIndexSetProvider} for MongoDB.
    + */
    +public class MongoPcjIndexSetProvider extends AbstractPcjIndexSetProvider {
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider. (not null)
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf) {
    +        super(conf);
    +    }
    +
    +    /**
    +     * Creates a new {@link MongoPcjIndexSetProvider}.
    +     * @param conf - The configuration for this provider.
    +     * @param indices - The predefined indicies on this provider.
    +     * @param client - The {@link MongoClient} used to connect to mongo.
    +     */
    +    public MongoPcjIndexSetProvider(final StatefulMongoDBRdfConfiguration conf, final List<ExternalTupleSet> indices) {
    +        super(conf, indices);
    +    }
    +
    +    @Override
    +    protected List<ExternalTupleSet> getIndices() throws PcjIndexSetException {
    +        try {
    +            //TODO use the PCJ collection.
    --- End diff --
    
    no, removed it


---

[GitHub] incubator-rya pull request #172: RYA-303 Mongo PCJ Support

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/172#discussion_r160834100
  
    --- Diff: extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/mongo/MongoPcjDocuments.java ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.storage.mongo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static java.util.Objects.requireNonNull;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.accumulo.core.security.Authorizations;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.model.VisibilityBindingSet;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.api.resolver.RyaToRdfConversions;
    +import org.apache.rya.api.utils.CloseableIterator;
    +import org.apache.rya.indexing.pcj.storage.PcjMetadata;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.bson.Document;
    +import org.bson.conversions.Bson;
    +import org.openrdf.model.URI;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.query.BindingSet;
    +import org.openrdf.query.MalformedQueryException;
    +import org.openrdf.query.QueryEvaluationException;
    +import org.openrdf.query.QueryLanguage;
    +import org.openrdf.query.TupleQuery;
    +import org.openrdf.query.TupleQueryResult;
    +import org.openrdf.query.impl.MapBindingSet;
    +import org.openrdf.repository.RepositoryConnection;
    +import org.openrdf.repository.RepositoryException;
    +
    +import com.mongodb.MongoClient;
    +import com.mongodb.client.FindIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Creates and modifies PCJs in MongoDB. PCJ's are stored as follows:
    + *
    + * <pre>
    + * <code>
    + * ----- PCJ Metadata Doc -----
    + * {
    + *   _id: [pcj_name]_METADATA,
    + *   sparql: [sparql query to match results],
    + *   varOrders: [varOrder1, VarOrder2, ..., VarOrdern]
    + *   cardinality: [number of results]
    + * }
    + *
    + * ----- PCJ Results Doc -----
    + * {
    + *   pcjId: [pcj_name],
    + *   visibilities: [visibilities]
    + *   [binding_var1]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + *   .
    + *   .
    + *   .
    + *   [binding_varn]: {
    + *     uri: [type_uri],
    + *     value: [value]
    + *   }
    + * }
    + * </code>
    + * </pre>
    + */
    +public class MongoPcjDocuments {
    +    public static final String PCJ_COLLECTION_NAME = "pcjs";
    +
    +    // metadata fields
    +    public static final String CARDINALITY_FIELD = "cardinality";
    +    public static final String SPARQL_FIELD = "sparql";
    +    public static final String PCJ_ID = "_id";
    +    public static final String VAR_ORDER_FIELD = "varOrders";
    +
    +    // pcj results fields
    +    private static final String BINDING_VALUE = "value";
    +    private static final String BINDING_TYPE = "rdfType";
    +    private static final String VISIBILITIES_FIELD = "visibilities";
    +    private static final String PCJ_NAME = "pcjId";
    +
    +    private final MongoCollection<Document> pcjCollection;
    +    private static final PcjVarOrderFactory pcjVarOrderFactory = new ShiftVarOrderFactory();
    +
    +    /**
    +     * Creates a new {@link MongoPcjDocuments}.
    +     * @param client - The {@link MongoClient} to use to connect to mongo.
    +     * @param ryaInstanceName - The rya instance to connect to.
    +     */
    +    public MongoPcjDocuments(final MongoClient client, final String ryaInstanceName) {
    +        requireNonNull(client);
    +        requireNonNull(ryaInstanceName);
    +        pcjCollection = client.getDatabase(ryaInstanceName).getCollection(PCJ_COLLECTION_NAME);
    +    }
    +
    +    private String makeMetadataID(final String pcjId) {
    +        return pcjId + "_METADATA";
    +    }
    +
    +    /**
    +     * Creates a {@link Document} containing the metadata defining the PCj.
    +     * 
    +     * @param pcjId
    +     *            - The name of the PCJ. (not null)
    --- End diff --
    
    done


---