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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query Support

GitHub user meiercaleb opened a pull request:

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

    RYA-273-Construct Query Support

    <!--
    Licensed to the Apache Software Foundation (ASF) under one
    or more contributor license agreements.  See the NOTICE file
    distributed with this work for additional information
    regarding copyright ownership.  The ASF licenses this file
    to you under the Apache License, Version 2.0 (the
    "License"); you may not use this file except in compliance
    with the License.  You may obtain a copy of the License at
    
      http://www.apache.org/licenses/LICENSE-2.0
    
    Unless required by applicable law or agreed to in writing,
    software distributed under the License is distributed on an
    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    KIND, either express or implied.  See the License for the
    specific language governing permissions and limitations
    under the License.
    -->
    ## Description
    Added construct support to Rya-Fluo application
    
    ### Tests
    Added integration tests to the pcj.fluo.integration test project.
    Tests ensure that the metadata tree is constructed successfully within
    the Fluo application and that construct statements are successfully created
    and exported to Kafka. 
    
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-273)
    
    ### Checklist
    - [ ] Code Review
    - [x] Squash Commits
    
    #### People To Review
    Andrew Smith, David Lotts


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

    $ git pull https://github.com/meiercaleb/incubator-rya RYA-273

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

    https://github.com/apache/incubator-rya/pull/161.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 #161
    
----
commit 7a4b45c36f27dde1e689179878ee1a9a60b22bc8
Author: Caleb Meier <ca...@parsons.com>
Date:   2017-04-15T02:20:25Z

    RYA-273-Construct Query Support

----


---
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 #161: RYA-273-Construct Query 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/161#discussion_r119748164
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporterFactory.java ---
    @@ -0,0 +1,50 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import org.apache.fluo.api.observer.Observer.Context;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporterFactory.ConfigurationException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporterFactory.IncrementalExporterFactoryException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporterFactory;
    +
    +import com.google.common.base.Optional;
    +
    +public class KafkaRyaSubGraphExporterFactory implements IncrementalRyaSubGraphExporterFactory {
    --- End diff --
    
    Docs


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122227312
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    --- End diff --
    
    Doh! Sorry, cut and paste job.  Didn't write this class.  Done.


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

[GitHub] incubator-rya issue #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/241/<h2>Failed Tests: <span class='status-failure'>8</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.export.integration' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.export.integration/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.export.integration</a>: <span class='status-failure'>1</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.export.integration/testReport/org.apache.rya.indexing.export/StoreToStoreIT/initializationError/'><strong>org.apache.rya.indexing.export.StoreToStoreIT.initializationError</strong></a></li></ul><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger' /><a href='https://builds.apache.org/job/incubator-rya-master-wi
 th-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger</a>: <span class='status-failure'>6</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTool/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTool</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyHierarchy/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyHierarchy</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySameA
 s/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySameAs</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTransitive/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTransitive</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyInverse/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyInverse</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySymmetry/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySymmetry</stro
 ng></a></li></ul><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.pcj.fluo.integration' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.pcj.fluo.integration/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.pcj.fluo.integration</a>: <span class='status-failure'>1</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/241/org.apache.rya$rya.pcj.fluo.integration/testReport/org.apache.rya.indexing.pcj.fluo.integration/KafkaRyaSubGraphExportIT/embeddedKafkaTest/'><strong>org.apache.rya.indexing.pcj.fluo.integration.KafkaRyaSubGraphExportIT.embeddedKafkaTest</strong></a></li></ul>



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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123320763
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/ConstructQueryMetadata.java ---
    @@ -0,0 +1,180 @@
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +/*
    + * 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.
    + */
    +import org.apache.commons.lang3.builder.EqualsBuilder;
    +import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119747120
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java ---
    @@ -227,9 +271,13 @@ public String withRyaIntegration(
             } catch (final IOException e) {
                 log.warn("Ignoring IOException thrown while closing the AccumuloRyaQueryEngine used by CreatePCJ.", e);
             }
    -
    -        // return queryId to the caller for later monitoring from the export.
    --- End diff --
    
    This doc is still accurate, right? Might as well leave it there.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r119747835
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/IncrementalRyaSubGraphExporter.java ---
    @@ -0,0 +1,39 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export;
    +/*
    + * 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.
    + */
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +
    +/**
    + * Incrementally exports RyaSubGraphs that are generated by SPARQL Construct Queries
    --- End diff --
    
    link to RyaSubGraph


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122229447
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    +        String subject = input.readString();
    +        String predicate = input.readString();
    +        String objectType = input.readString();
    +        String objectValue = input.readString();
    +        RyaType value;
    +        if (objectType.equals("http://www.w3.org/2001/XMLSchema#anyURI")){
    +            value = new RyaURI(objectValue);
    +        }
    +        else {
    +            value = new RyaType(new URIImpl(objectType), objectValue);
    +        }
    +        RyaStatement statement = new RyaStatement(new RyaURI(subject), new RyaURI(predicate), value);
    +        int length = 0;
    +        boolean hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setContext(new RyaURI(input.readString()));
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            length = input.readInt();
    +            statement.setColumnVisibility(input.readBytes(length));
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setQualifer(input.readString());
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setTimestamp(input.readLong());
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            length = input.readInt();
    +            statement.setValue(input.readBytes(length));
    +        }
    +
    +        return statement;
    +    }
    +
    +    public static RyaStatement read(Input input){
    +        String subject = input.readString();
    +        String predicate = input.readString();
    +        String objectType = input.readString();
    +        String objectValue = input.readString();
    +        RyaType value;
    +        if (objectType.equals("http://www.w3.org/2001/XMLSchema#anyURI")){
    --- End diff --
    
    Yes.  Done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119746146
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaSubGraphSerializer.java ---
    @@ -0,0 +1,62 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +public class RyaSubGraphSerializer extends Serializer<RyaSubGraph>{
    --- End diff --
    
    Docs.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122720322
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    +
    +    private Set<ConstructProjection> projections;
    +    private Set<String> bNodeNames;
    +    
    +    public ConstructGraph(Set<ConstructProjection> projections) {
    +        Preconditions.checkNotNull(projections);
    --- End diff --
    
    I don't think this was officially decided.  Maybe you and Kevin agreed on this convention, but I don't remember having the conversation.  Does is really matter if we're using a guava dependency in either case?


---
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 #161: RYA-273-Construct Query 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/161#discussion_r123546312
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/ConstructQueryMetadata.java ---
    @@ -0,0 +1,192 @@
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +/*
    + * 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.
    + */
    +import org.apache.commons.lang3.builder.EqualsBuilder;
    +import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Metadata object used to store metadata for Construct Query Nodes found in
    + * SPARQL queries.
    + *
    + */
    +public class ConstructQueryMetadata extends CommonNodeMetadata {
    +
    +    private String childNodeId;
    +    private ConstructGraph graph;
    +    private String sparql;
    +
    +    /**
    +     * Creates ConstructQueryMetadata object from the provided metadata arguments.
    +     * @param nodeId - id for the ConstructQueryNode
    +     * @param childNodeId - id for the child of the ConstructQueryNode
    +     * @param graph - {@link ConstructGraph} used to project {@link BindingSet}s onto sets of statement representing construct graph
    +     * @param sparql - SPARQL query containing construct graph
    +     */
    +    public ConstructQueryMetadata(String nodeId, String childNodeId, ConstructGraph graph, String sparql) {
    +        super(nodeId, new VariableOrder("subject", "predicate", "object"));
    +        Preconditions.checkNotNull(childNodeId);
    +        Preconditions.checkNotNull(graph);
    +        Preconditions.checkNotNull(sparql);
    +        this.childNodeId = childNodeId;
    +        this.graph = graph;
    +        this.sparql = sparql;
    +    }
    +
    +    /**
    +     * @return sparql query string representing this construct query
    +     */
    +    public String getSparql() {
    +        return sparql;
    +    }
    +
    +    /**
    +     * @return The node whose results are projected onto the given
    +     *         {@link ConstructGraph}.
    +     */
    +    public String getChildNodeId() {
    +        return childNodeId;
    +    }
    +
    +    /**
    +     * @return The ConstructGraph used to form statement {@link BindingSet}s for
    +     *         this Construct Query
    +     */
    +    public ConstructGraph getConstructGraph() {
    +        return graph;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(super.getNodeId(), super.getVariableOrder(), childNodeId, graph, sparql);
    +    }
    +
    +    @Override
    +    public boolean equals(final Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +
    +        if (o instanceof ConstructQueryMetadata) {
    +            ConstructQueryMetadata queryMetadata = (ConstructQueryMetadata) o;
    +            if (super.equals(queryMetadata)) {
    +                return new EqualsBuilder().append(childNodeId, queryMetadata.childNodeId).append(graph, queryMetadata.graph)
    +                        .append(sparql, queryMetadata.sparql).isEquals();
    +            }
    +            return false;
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return new StringBuilder().append("Construct Query Metadata {\n").append("    Node ID: " + super.getNodeId() + "\n")
    --- End diff --
    
    Okay. I'll keep it in mind in the future.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122720445
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +/**
    + * Kryo Serializer for {@link RyaStatement}s
    + *
    + */
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    /**
    +     * Uses Kryo to read a RyaStatement from {@link Input}
    +     * @param kryo - reads statement from input
    +     * @param input - Input stream that statement is read from
    +     * @param type - Type read from input stream
    +     * @return - statement read from input stream
    +     */
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    +        return read(input);
    +    }
    +
    +    /**
    +     * Reads RyaStatement from {@link Input} stream
    +     * @param input - input stream that statement is read from
    +     * @return - statement read from input stream
    +     */
    +    public static RyaStatement read(Input input){
    +        String subject = input.readString();
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122717901
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    +        this.id = id;
    +        this.statements = new HashSet<>();
    +    }
    +    
    +    public RyaSubGraph(String id, Set<RyaStatement> statements) {
    +        this.id = id;
    +        this.statements = statements;
    +    }
    +
    +    /**
    +     * @return id of this subgraph
    +     */
    +    public String getId() {
    +        return id;
    +    }
    +    
    +    /**
    +     * @return RyaStatements representing this subgraph
    +     */
    +    public Set<RyaStatement> getStatements() {
    +        return statements;
    +    }
    +    
    +    /**
    +     * Sets id of subgraph
    +     * @param id - id of subgraph
    +     */
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +    
    +    public void setStatements(Set<RyaStatement> statements) {
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122248040
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporterFactory.java ---
    @@ -0,0 +1,50 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import org.apache.fluo.api.observer.Observer.Context;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporterFactory.ConfigurationException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporterFactory.IncrementalExporterFactoryException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporterFactory;
    +
    +import com.google.common.base.Optional;
    +
    +public class KafkaRyaSubGraphExporterFactory implements IncrementalRyaSubGraphExporterFactory {
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122229499
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaSubGraphSerializer.java ---
    @@ -0,0 +1,62 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +public class RyaSubGraphSerializer extends Serializer<RyaSubGraph>{
    --- End diff --
    
    Done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123094218
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporter.java ---
    @@ -0,0 +1,81 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Exports {@link RyaSubGraph}s to Kafka from Rya Fluo Application 
    + *
    + */
    +public class KafkaRyaSubGraphExporter implements IncrementalRyaSubGraphExporter {
    +
    +    private final KafkaProducer<String, RyaSubGraph> producer;
    +    private static final Logger log = Logger.getLogger(KafkaRyaSubGraphExporter.class);
    +
    +    public KafkaRyaSubGraphExporter(KafkaProducer<String, RyaSubGraph> producer) {
    +        Preconditions.checkNotNull(producer);
    --- End diff --
    
    you have this statically imported


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122221110
  
    --- Diff: common/rya.api/pom.xml ---
    @@ -70,7 +70,11 @@ under the License.
                 <groupId>com.github.stephenc.jcip</groupId>
                 <artifactId>jcip-annotations</artifactId>
             </dependency>
    -        
    +        <dependency>
    --- End diff --
    
    Done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123321448
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java ---
    @@ -417,6 +438,60 @@ public void meet(final Projection node) {
                 // Walk to the next node.
                 super.meet(node);
             }
    +        
    +        
    +        public void meet(Reduced node) {
    +            //create id, initialize ConstructQueryMetadata builder, register ConstructQueryMetadata 
    +            //builder with FluoQueryBuilder, and add metadata that we currently have
    +            final String constructId = nodeIds.getOrMakeId(node);
    +            final ConstructQueryMetadata.Builder constructBuilder = ConstructQueryMetadata.builder();
    +            constructBuilder.setNodeId(constructId);
    +            fluoQueryBuilder.setConstructQueryMetadata(constructBuilder);
    +            constructBuilder.setSparql(sparql);
    +            
    +            //get child node
    +            QueryModelNode child = node.getArg();
    +            Preconditions.checkArgument(child instanceof Projection || child instanceof MultiProjection);
    --- End diff --
    
    done.


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

[GitHub] incubator-rya issue #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/236/<h2>Build result: ABORTED</span></h2>[...truncated 10.96 MB...]	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)	at java.lang.Thread.run(Thread.java:748)Build timed out (after 180 minutes). Marking the build as aborted.Results :Tests run: 43, Failures: 0, Errors: 0, Skipped: 0[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 24 iota=54[INFO] [INFO] --- maven-failsafe-plugin:2.18.1:verify (default) @ rya.pcj.fluo.integration ---[INFO] Failsafe report directory: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.integration/target/failsafe-reports[ERROR] There was a timeout or other error in the fork[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Inval
 id object ID 24 iota=54[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 24 iota=54[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO]                                                                         [INFO] ------------------------------------------------------------------------[INFO] Building Apache Rya PCJ Fluo Demo 3.2.11-incubating-SNAPSHOT[INFO] ------------------------------------------------------------------------Build was abortedchannel stoppedSetting status of 4dd53b12cf69f70a67997e6ec59c845631ad78f1 to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/236/ 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 #161: RYA-273-Construct Query 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/161#discussion_r123319147
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructQueryResultUpdater.java ---
    @@ -0,0 +1,72 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSchema;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.kafka.RyaSubGraphKafkaSerDe;
    +import org.apache.rya.indexing.pcj.fluo.app.query.ConstructQueryMetadata;
    +import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public class ConstructQueryResultUpdater {
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122254184
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    +
    +    private Set<ConstructProjection> projections;
    +    private Set<String> bNodeNames;
    +    
    +    public ConstructGraph(Set<ConstructProjection> projections) {
    --- End diff --
    
    docs


---
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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/237/<h2>Failed Tests: <span class='status-failure'>10</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger</a>: <span class='status-failure'>6</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTool/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTool</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyHierarch
 y/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyHierarchy</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySameAs/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySameAs</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTransitive/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTransitive</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyInverse/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyInverse</stron
 g></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySymmetry/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySymmetry</strong></a></li></ul><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.pcj.fluo.integration' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.pcj.fluo.integration/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.pcj.fluo.integration</a>: <span class='status-failure'>4</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.pcj.fluo.integration/testReport/org.apache.rya.indexing.pcj.fluo.integration/KafkaRyaSubGraphExportIT/constructQueryWithBlankNodesAndMultipleSubGraphs/'><strong>org.apach
 e.rya.indexing.pcj.fluo.integration.KafkaRyaSubGraphExportIT.constructQueryWithBlankNodesAndMultipleSubGraphs</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.pcj.fluo.integration/testReport/org.apache.rya.indexing.pcj.fluo.integration/KafkaRyaSubGraphExportIT/basicConstructQuery/'><strong>org.apache.rya.indexing.pcj.fluo.integration.KafkaRyaSubGraphExportIT.basicConstructQuery</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.pcj.fluo.integration/testReport/org.apache.rya.indexing.pcj.fluo.integration/KafkaRyaSubGraphExportIT/basicConstructQueryWithVis/'><strong>org.apache.rya.indexing.pcj.fluo.integration.KafkaRyaSubGraphExportIT.basicConstructQueryWithVis</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/237/org.apache.rya$rya.pcj.fluo.integration/testRepor
 t/org.apache.rya.indexing.pcj.fluo.integration/KafkaRyaSubGraphExportIT/constructQueryWithVisAndMultipleSubGraphs/'><strong>org.apache.rya.indexing.pcj.fluo.integration.KafkaRyaSubGraphExportIT.constructQueryWithVisAndMultipleSubGraphs</strong></a></li></ul>



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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122247000
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporter.java ---
    @@ -0,0 +1,69 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +
    +import jline.internal.Preconditions;
    +
    +public class KafkaRyaSubGraphExporter implements IncrementalRyaSubGraphExporter {
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122243955
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java ---
    @@ -227,9 +271,13 @@ public String withRyaIntegration(
             } catch (final IOException e) {
                 log.warn("Ignoring IOException thrown while closing the AccumuloRyaQueryEngine used by CreatePCJ.", e);
             }
    -
    -        // return queryId to the caller for later monitoring from the export.
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123092575
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructQueryResultUpdater.java ---
    @@ -0,0 +1,72 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSchema;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.kafka.RyaSubGraphKafkaSerDe;
    +import org.apache.rya.indexing.pcj.fluo.app.query.ConstructQueryMetadata;
    +import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public class ConstructQueryResultUpdater {
    --- End diff --
    
    docs


---
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 #161: RYA-273-Construct Query 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/161#discussion_r119745525
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    --- End diff --
    
    Needs docs.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122721817
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    +
    +    private Set<ConstructProjection> projections;
    +    private Set<String> bNodeNames;
    +    
    +    public ConstructGraph(Set<ConstructProjection> projections) {
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122222068
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,88 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +public class RyaSubGraph {
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122224102
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119745766
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    +        String subject = input.readString();
    +        String predicate = input.readString();
    +        String objectType = input.readString();
    +        String objectValue = input.readString();
    +        RyaType value;
    +        if (objectType.equals("http://www.w3.org/2001/XMLSchema#anyURI")){
    +            value = new RyaURI(objectValue);
    +        }
    +        else {
    +            value = new RyaType(new URIImpl(objectType), objectValue);
    +        }
    +        RyaStatement statement = new RyaStatement(new RyaURI(subject), new RyaURI(predicate), value);
    +        int length = 0;
    +        boolean hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setContext(new RyaURI(input.readString()));
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            length = input.readInt();
    +            statement.setColumnVisibility(input.readBytes(length));
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setQualifer(input.readString());
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            statement.setTimestamp(input.readLong());
    +        }
    +        hasNextValue = input.readBoolean();
    +        if (hasNextValue){
    +            length = input.readInt();
    +            statement.setValue(input.readBytes(length));
    +        }
    +
    +        return statement;
    +    }
    +
    +    public static RyaStatement read(Input input){
    +        String subject = input.readString();
    +        String predicate = input.readString();
    +        String objectType = input.readString();
    +        String objectValue = input.readString();
    +        RyaType value;
    +        if (objectType.equals("http://www.w3.org/2001/XMLSchema#anyURI")){
    --- End diff --
    
    Is there a constant for this URI somewhere?


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122251387
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    +        this.id = id;
    +        this.statements = new HashSet<>();
    +    }
    +    
    +    public RyaSubGraph(String id, Set<RyaStatement> statements) {
    +        this.id = id;
    +        this.statements = statements;
    +    }
    +
    +    /**
    +     * @return id of this subgraph
    +     */
    +    public String getId() {
    +        return id;
    +    }
    +    
    +    /**
    +     * @return RyaStatements representing this subgraph
    +     */
    +    public Set<RyaStatement> getStatements() {
    +        return statements;
    +    }
    +    
    +    /**
    +     * Sets id of subgraph
    +     * @param id - id of subgraph
    +     */
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +    
    +    public void setStatements(Set<RyaStatement> statements) {
    --- End diff --
    
    and here


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122244695
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/IncrementalRyaSubGraphExporter.java ---
    @@ -0,0 +1,39 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export;
    +/*
    + * 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.
    + */
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +
    +/**
    + * Incrementally exports RyaSubGraphs that are generated by SPARQL Construct Queries
    --- End diff --
    
    done



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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123319505
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporter.java ---
    @@ -0,0 +1,81 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Exports {@link RyaSubGraph}s to Kafka from Rya Fluo Application 
    + *
    + */
    +public class KafkaRyaSubGraphExporter implements IncrementalRyaSubGraphExporter {
    +
    +    private final KafkaProducer<String, RyaSubGraph> producer;
    +    private static final Logger log = Logger.getLogger(KafkaRyaSubGraphExporter.class);
    +
    +    public KafkaRyaSubGraphExporter(KafkaProducer<String, RyaSubGraph> producer) {
    +        Preconditions.checkNotNull(producer);
    --- End diff --
    
    I do indeed.  Done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119745391
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,88 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +public class RyaSubGraph {
    --- End diff --
    
    Needs docs.


---
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 #161: RYA-273-Construct Query Support

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

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


---
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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/239/<h2>Failed Tests: <span class='status-failure'>6</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger</a>: <span class='status-failure'>6</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTool/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTool</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyHierarchy
 /'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyHierarchy</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySameAs/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySameAs</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTransitive/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTransitive</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyInverse/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyInverse</strong
 ></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/239/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySymmetry/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySymmetry</strong></a></li></ul>



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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122251690
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +/**
    + * Kryo Serializer for {@link RyaStatement}s
    + *
    + */
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    /**
    +     * Uses Kryo to read a RyaStatement from {@link Input}
    +     * @param kryo - reads statement from input
    +     * @param input - Input stream that statement is read from
    +     * @param type - Type read from input stream
    +     * @return - statement read from input stream
    +     */
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    +        return read(input);
    +    }
    +
    +    /**
    +     * Reads RyaStatement from {@link Input} stream
    +     * @param input - input stream that statement is read from
    +     * @return - statement read from input stream
    +     */
    +    public static RyaStatement read(Input input){
    +        String subject = input.readString();
    --- End diff --
    
    null check


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119746096
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    +        output.writeString(object.getPredicate().getData());
    +        output.writeString(object.getObject().getDataType().toString());
    +        output.writeString(object.getObject().getData());
    +        boolean hasContext = object.getContext() != null;
    +        output.writeBoolean(hasContext);
    +        if(hasContext){
    +            output.writeString(object.getContext().getData());
    +        }
    +        boolean shouldWrite = object.getColumnVisibility() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeInt(object.getColumnVisibility().length);
    +            output.writeBytes(object.getColumnVisibility());
    +        }
    +        shouldWrite = object.getQualifer() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeString(object.getQualifer());
    +        }
    +        shouldWrite = object.getTimestamp() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeLong(object.getTimestamp());
    +        }
    +        shouldWrite = object.getValue() != null;
    +        output.writeBoolean(shouldWrite);
    +        if(shouldWrite){
    +            output.writeBytes(object.getValue());
    +        }
    +    }   
    +
    +    @Override
    +    public void write(Kryo kryo, Output output, RyaStatement object) {
    +        writeToKryo(kryo, output, object);
    +    }
    +    
    +    public static RyaStatement readFromKryo(Kryo kryo, Input input, Class<RyaStatement> type){
    --- End diff --
    
    This method an public static RyaStatement read(Input input) look identical. Why not just delegate to the other one since I think you're ignoring the kryo and type parameters.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122719074
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    +        this.id = id;
    +        this.statements = new HashSet<>();
    +    }
    +    
    +    public RyaSubGraph(String id, Set<RyaStatement> statements) {
    +        this.id = id;
    +        this.statements = statements;
    +    }
    +
    +    /**
    +     * @return id of this subgraph
    +     */
    +    public String getId() {
    +        return id;
    +    }
    +    
    +    /**
    +     * @return RyaStatements representing this subgraph
    +     */
    +    public Set<RyaStatement> getStatements() {
    +        return statements;
    +    }
    +    
    +    /**
    +     * Sets id of subgraph
    +     * @param id - id of subgraph
    +     */
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +    
    +    public void setStatements(Set<RyaStatement> statements) {
    +        this.statements = statements;
    +    }
    +    
    +
    +    /**
    +     * Adds statement to this subgraph
    +     * @param statement - RyaStatement to be added to subgraph
    +     */
    +    public void addStatement(RyaStatement statement){
    +        statements.add(statement);
    +    }
    +    
    +    @Override
    +    public boolean equals(Object other) {
    +        
    +        if(this == other) {
    +            return true;
    +        }
    +        
    +        if(other instanceof RyaSubGraph) {
    +            RyaSubGraph bundle = (RyaSubGraph) other;
    +            return Objects.equal(this.id, ((RyaSubGraph) other).id) && Objects.equal(this.statements,bundle.statements);
    --- End diff --
    
    Don't see how this is possible.  Objects.equal() returns a boolean.  How can you chain method calls in the above fashion?


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122720810
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeletePcj.java ---
    @@ -229,18 +236,20 @@ private void deletePcjIdAndSparqlMetadata(final Transaction tx, final String pcj
     
     
         /**
    -     * Deletes all BindingSets associated with the specified nodeId.
    +     * Deletes all results (BindingSets or Statements) associated with the specified nodeId.
          *
    -     * @param nodeId - nodeId whose {@link BindingSet}s will be deleted. (not null)
    -     * @param client - Used to delete the data. (not null)
    +     * @param nodeId
    --- End diff --
    
    done


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

[GitHub] incubator-rya issue #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/245/<h2>Build result: ABORTED</span></h2>[...truncated 6.53 MB...][INFO] [INFO] --- maven-jar-plugin:2.5:jar (default-jar) @ rya.pcj.fluo.demo ---[INFO] Building jar: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/target/rya.pcj.fluo.demo-3.2.11-incubating-SNAPSHOT.jar[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-site-plugin:3.4:attach-descriptor (attach-descriptor) @ rya.pcj.fluo.demo ---[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-source-plugin:2.4:jar-no-fork (attach-sources) @ rya.pcj.fluo.demo ---[INFO] Building jar: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-op
 tionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/target/rya.pcj.fluo.demo-3.2.11-incubating-SNAPSHOT-sources.jar[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-jar-plugin:2.5:test-jar (default) @ rya.pcj.fluo.demo ---[INFO] Building jar: /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/target/rya.pcj.fluo.demo-3.2.11-incubating-SNAPSHOT-tests.jar[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-assembly-plugin:2.4.1:single (make-assembly) @ rya.pcj.fluo.demo ---Adding org.apache.maven.scm.provider.ScmProvidersvn[INFO] Building jar: /home/jenkins/jenkins-slave/workspace/
 incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/target/rya.pcj.fluo.demo-3.2.11-incubating-SNAPSHOT-jar-with-dependencies.jarBuild was abortedchannel stoppedSetting status of 872e16732e936c116845db9de21ec9d8934a1f3e to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/245/ 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 #161: RYA-273-Construct Query 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/161#discussion_r122254149
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    --- End diff --
    
    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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161#discussion_r123325067
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/ConstructQueryMetadata.java ---
    @@ -0,0 +1,192 @@
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +/*
    + * 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.
    + */
    +import org.apache.commons.lang3.builder.EqualsBuilder;
    +import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Metadata object used to store metadata for Construct Query Nodes found in
    + * SPARQL queries.
    + *
    + */
    +public class ConstructQueryMetadata extends CommonNodeMetadata {
    +
    +    private String childNodeId;
    +    private ConstructGraph graph;
    +    private String sparql;
    +
    +    /**
    +     * Creates ConstructQueryMetadata object from the provided metadata arguments.
    +     * @param nodeId - id for the ConstructQueryNode
    +     * @param childNodeId - id for the child of the ConstructQueryNode
    +     * @param graph - {@link ConstructGraph} used to project {@link BindingSet}s onto sets of statement representing construct graph
    +     * @param sparql - SPARQL query containing construct graph
    +     */
    +    public ConstructQueryMetadata(String nodeId, String childNodeId, ConstructGraph graph, String sparql) {
    +        super(nodeId, new VariableOrder("subject", "predicate", "object"));
    +        Preconditions.checkNotNull(childNodeId);
    +        Preconditions.checkNotNull(graph);
    +        Preconditions.checkNotNull(sparql);
    +        this.childNodeId = childNodeId;
    +        this.graph = graph;
    +        this.sparql = sparql;
    +    }
    +
    +    /**
    +     * @return sparql query string representing this construct query
    +     */
    +    public String getSparql() {
    +        return sparql;
    +    }
    +
    +    /**
    +     * @return The node whose results are projected onto the given
    +     *         {@link ConstructGraph}.
    +     */
    +    public String getChildNodeId() {
    +        return childNodeId;
    +    }
    +
    +    /**
    +     * @return The ConstructGraph used to form statement {@link BindingSet}s for
    +     *         this Construct Query
    +     */
    +    public ConstructGraph getConstructGraph() {
    +        return graph;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(super.getNodeId(), super.getVariableOrder(), childNodeId, graph, sparql);
    +    }
    +
    +    @Override
    +    public boolean equals(final Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +
    +        if (o instanceof ConstructQueryMetadata) {
    +            ConstructQueryMetadata queryMetadata = (ConstructQueryMetadata) o;
    +            if (super.equals(queryMetadata)) {
    +                return new EqualsBuilder().append(childNodeId, queryMetadata.childNodeId).append(graph, queryMetadata.graph)
    +                        .append(sparql, queryMetadata.sparql).isEquals();
    +            }
    +            return false;
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return new StringBuilder().append("Construct Query Metadata {\n").append("    Node ID: " + super.getNodeId() + "\n")
    --- End diff --
    
    Consider Objects.ToStringHelper


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122251594
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +/**
    + * Kryo Serializer for {@link RyaStatement}s
    + *
    + */
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    --- End diff --
    
    null checks?


---
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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/238/<h2>Build result: ABORTED</span></h2>[...truncated 6.52 MB...][INFO] --- maven-compiler-plugin:3.2:compile (default-compile) @ rya.pcj.fluo.demo ---[INFO] Changes detected - recompiling the module![INFO] Compiling 3 source files to /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/target/classes[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-resources-plugin:2.7:testResources (default-testResources) @ rya.pcj.fluo.demo ---[INFO] Using 'UTF-8' encoding to copy filtered resources.[INFO] skip non existing resourceDirectory /home/jenkins/jenkins-slave/workspace/incubator-rya-master-with-optionals-pull-requests/extras/rya.pcj.fluo/pcj.fluo.demo/src/test/resources[INFO] Copying 3 resources[WARNING] Failed to notify
  spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-compiler-plugin:3.2:testCompile (default-testCompile) @ rya.pcj.fluo.demo ---[INFO] No sources to compile[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- maven-surefire-plugin:2.18.1:test (default-test) @ rya.pcj.fluo.demo ---[WARNING] Failed to notify spy hudson.maven.Maven3Builder$JenkinsEventSpy: java.util.concurrent.ExecutionException: Invalid object ID 12 iota=54[INFO] [INFO] --- animal-sniffer-maven-plugin:1.15:check (default) @ rya.pcj.fluo.demo ---[INFO] Checking unresolved references to org.codehaus.mojo.signature:java18:1.0Build was abortedchannel stoppedSetting status of ae65c0ab747e8f25812c95defe778cbb581c674d to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/238/ and mes
 sage: '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 #161: RYA-273-Construct Query 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/161#discussion_r122251341
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    --- End diff --
    
    don't forget to doc these guys too


---
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 #161: RYA-273-Construct Query 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/161#discussion_r123094972
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java ---
    @@ -417,6 +438,60 @@ public void meet(final Projection node) {
                 // Walk to the next node.
                 super.meet(node);
             }
    +        
    +        
    +        public void meet(Reduced node) {
    +            //create id, initialize ConstructQueryMetadata builder, register ConstructQueryMetadata 
    +            //builder with FluoQueryBuilder, and add metadata that we currently have
    +            final String constructId = nodeIds.getOrMakeId(node);
    +            final ConstructQueryMetadata.Builder constructBuilder = ConstructQueryMetadata.builder();
    +            constructBuilder.setNodeId(constructId);
    +            fluoQueryBuilder.setConstructQueryMetadata(constructBuilder);
    +            constructBuilder.setSparql(sparql);
    +            
    +            //get child node
    +            QueryModelNode child = node.getArg();
    +            Preconditions.checkArgument(child instanceof Projection || child instanceof MultiProjection);
    --- End diff --
    
    you seem to be conflicted about statically importing these or not....


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122720389
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/serialization/kryo/RyaStatementSerializer.java ---
    @@ -0,0 +1,156 @@
    +package org.apache.rya.api.domain.serialization.kryo;
    +/*
    + * 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.
    + */
    +import org.openrdf.model.impl.URIImpl;
    +import org.openrdf.model.vocabulary.XMLSchema;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.domain.RyaURI;
    +
    +/**
    + * Kryo Serializer for {@link RyaStatement}s
    + *
    + */
    +public class RyaStatementSerializer extends Serializer<RyaStatement> {
    +    
    +    /**
    +     * Uses Kryo to write RyaStatement to {@lin Output}
    +     * @param kryo - writes statement to output
    +     * @param output - output stream that statement is written to
    +     * @param object - statement written to output
    +     */
    +    public static void writeToKryo(Kryo kryo, Output output, RyaStatement object) {
    +        output.writeString(object.getSubject().getData());
    --- End diff --
    
    done.


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

[GitHub] incubator-rya issue #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    
    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/242/<h2>Failed Tests: <span class='status-failure'>6</span></h2><h3><a name='incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger' /><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport'>incubator-rya-master-with-optionals-pull-requests/org.apache.rya:rya.merger</a>: <span class='status-failure'>6</span></h3><ul><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTool/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTool</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyHierarchy
 /'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyHierarchy</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySameAs/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySameAs</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyTransitive/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyTransitive</strong></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopyInverse/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopyInverse</strong
 ></a></li><li><a href='https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/242/org.apache.rya$rya.merger/testReport/org.apache.rya.accumulo.mr.merge/RulesetCopyIT/testRulesetCopySymmetry/'><strong>org.apache.rya.accumulo.mr.merge.RulesetCopyIT.testRulesetCopySymmetry</strong></a></li></ul>



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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122251503
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    +        this.id = id;
    +        this.statements = new HashSet<>();
    +    }
    +    
    +    public RyaSubGraph(String id, Set<RyaStatement> statements) {
    +        this.id = id;
    +        this.statements = statements;
    +    }
    +
    +    /**
    +     * @return id of this subgraph
    +     */
    +    public String getId() {
    +        return id;
    +    }
    +    
    +    /**
    +     * @return RyaStatements representing this subgraph
    +     */
    +    public Set<RyaStatement> getStatements() {
    +        return statements;
    +    }
    +    
    +    /**
    +     * Sets id of subgraph
    +     * @param id - id of subgraph
    +     */
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +    
    +    public void setStatements(Set<RyaStatement> statements) {
    +        this.statements = statements;
    +    }
    +    
    +
    +    /**
    +     * Adds statement to this subgraph
    +     * @param statement - RyaStatement to be added to subgraph
    +     */
    +    public void addStatement(RyaStatement statement){
    +        statements.add(statement);
    +    }
    +    
    +    @Override
    +    public boolean equals(Object other) {
    +        
    +        if(this == other) {
    +            return true;
    +        }
    +        
    +        if(other instanceof RyaSubGraph) {
    +            RyaSubGraph bundle = (RyaSubGraph) other;
    +            return Objects.equal(this.id, ((RyaSubGraph) other).id) && Objects.equal(this.statements,bundle.statements);
    --- End diff --
    
    you should be able to do Objects.equal().equals().equal().....


---
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 #161: RYA-273-Construct Query 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/161#discussion_r119748235
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/RyaSubGraphKafkaSerDe.java ---
    @@ -0,0 +1,69 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import java.io.ByteArrayOutputStream;
    +import java.util.Map;
    +
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.Serializer;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.api.domain.serialization.kryo.RyaSubGraphSerializer;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +public class RyaSubGraphKafkaSerDe implements Serializer<RyaSubGraph>, Deserializer<RyaSubGraph> {
    --- End diff --
    
    Docs


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122254058
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeletePcj.java ---
    @@ -229,18 +236,20 @@ private void deletePcjIdAndSparqlMetadata(final Transaction tx, final String pcj
     
     
         /**
    -     * Deletes all BindingSets associated with the specified nodeId.
    +     * Deletes all results (BindingSets or Statements) associated with the specified nodeId.
          *
    -     * @param nodeId - nodeId whose {@link BindingSet}s will be deleted. (not null)
    -     * @param client - Used to delete the data. (not null)
    +     * @param nodeId
    --- End diff --
    
    move these to same line


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122254262
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    +
    +    private Set<ConstructProjection> projections;
    +    private Set<String> bNodeNames;
    +    
    +    public ConstructGraph(Set<ConstructProjection> projections) {
    +        Preconditions.checkNotNull(projections);
    --- End diff --
    
    I think we're using Objects.requireNonNull() now?


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122249638
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/RyaSubGraphKafkaSerDe.java ---
    @@ -0,0 +1,69 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import java.io.ByteArrayOutputStream;
    +import java.util.Map;
    +
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.Serializer;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.api.domain.serialization.kryo.RyaSubGraphSerializer;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +
    +public class RyaSubGraphKafkaSerDe implements Serializer<RyaSubGraph>, Deserializer<RyaSubGraph> {
    --- End diff --
    
    done.


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r119745357
  
    --- Diff: common/rya.api/pom.xml ---
    @@ -70,7 +70,11 @@ under the License.
                 <groupId>com.github.stephenc.jcip</groupId>
                 <artifactId>jcip-annotations</artifactId>
             </dependency>
    -        
    +        <dependency>
    --- End diff --
    
    tabs.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r119748047
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaRyaSubGraphExporter.java ---
    @@ -0,0 +1,69 @@
    +package org.apache.rya.indexing.pcj.fluo.app.export.kafka;
    +/*
    + * 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.
    + */
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaSubGraph;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalBindingSetExporter.ResultExportException;
    +import org.apache.rya.indexing.pcj.fluo.app.export.IncrementalRyaSubGraphExporter;
    +
    +import jline.internal.Preconditions;
    +
    +public class KafkaRyaSubGraphExporter implements IncrementalRyaSubGraphExporter {
    --- End diff --
    
    Docs


---
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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161
  
    forgot to hit submit review


---
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 #161: RYA-273-Construct Query 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/161#discussion_r122717329
  
    --- Diff: common/rya.api/src/main/java/org/apache/rya/api/domain/RyaSubGraph.java ---
    @@ -0,0 +1,105 @@
    +package org.apache.rya.api.domain;
    +/*
    + * 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.
    + */
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * This class packages together a collection of {@link RyaStatement}s to form a subgraph
    + */
    +public class RyaSubGraph {
    +
    +    private String id;
    +    private Set<RyaStatement> statements;
    +    
    +    public RyaSubGraph(String id) {
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r122721000
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ConstructGraph.java ---
    @@ -0,0 +1,135 @@
    +package org.apache.rya.indexing.pcj.fluo.app;
    +/*
    + * 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.
    + */
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.BNode;
    +import org.openrdf.model.Value;
    +import org.openrdf.model.impl.BNodeImpl;
    +import org.openrdf.query.algebra.StatementPattern;
    +import org.openrdf.query.algebra.Var;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Creates a construct query graph (represented as a Set of
    + * {@link RyaStatement}s with Binding names subject, predicate, object) from a
    + * given BindingSet and the underlying {@link ConstructProjection}s.
    + *
    + */
    +public class ConstructGraph {
    +
    --- End diff --
    
    done


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

[GitHub] incubator-rya pull request #161: RYA-273-Construct Query 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/161#discussion_r123321422
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java ---
    @@ -417,6 +438,60 @@ public void meet(final Projection node) {
                 // Walk to the next node.
                 super.meet(node);
             }
    +        
    +        
    +        public void meet(Reduced node) {
    +            //create id, initialize ConstructQueryMetadata builder, register ConstructQueryMetadata 
    +            //builder with FluoQueryBuilder, and add metadata that we currently have
    +            final String constructId = nodeIds.getOrMakeId(node);
    +            final ConstructQueryMetadata.Builder constructBuilder = ConstructQueryMetadata.builder();
    +            constructBuilder.setNodeId(constructId);
    +            fluoQueryBuilder.setConstructQueryMetadata(constructBuilder);
    +            constructBuilder.setSparql(sparql);
    +            
    +            //get child node
    +            QueryModelNode child = node.getArg();
    +            Preconditions.checkArgument(child instanceof Projection || child instanceof MultiProjection);
    --- End diff --
    
    Think I just forgot that I had statically imported checkNotNull.  Didn't look to see if I was consistent.


---
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 #161: RYA-273-Construct Query 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/161#discussion_r123094575
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/ConstructQueryMetadata.java ---
    @@ -0,0 +1,180 @@
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +/*
    + * 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.
    + */
    +import org.apache.commons.lang3.builder.EqualsBuilder;
    +import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    --- End diff --
    
    more docs!


---
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 #161: RYA-273-Construct Query Support

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

    https://github.com/apache/incubator-rya/pull/161#discussion_r123325785
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/ConstructQueryMetadata.java ---
    @@ -0,0 +1,192 @@
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +/*
    + * 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.
    + */
    +import org.apache.commons.lang3.builder.EqualsBuilder;
    +import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.base.Preconditions;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Metadata object used to store metadata for Construct Query Nodes found in
    + * SPARQL queries.
    + *
    + */
    +public class ConstructQueryMetadata extends CommonNodeMetadata {
    +
    +    private String childNodeId;
    +    private ConstructGraph graph;
    +    private String sparql;
    +
    +    /**
    +     * Creates ConstructQueryMetadata object from the provided metadata arguments.
    +     * @param nodeId - id for the ConstructQueryNode
    +     * @param childNodeId - id for the child of the ConstructQueryNode
    +     * @param graph - {@link ConstructGraph} used to project {@link BindingSet}s onto sets of statement representing construct graph
    +     * @param sparql - SPARQL query containing construct graph
    +     */
    +    public ConstructQueryMetadata(String nodeId, String childNodeId, ConstructGraph graph, String sparql) {
    +        super(nodeId, new VariableOrder("subject", "predicate", "object"));
    +        Preconditions.checkNotNull(childNodeId);
    +        Preconditions.checkNotNull(graph);
    +        Preconditions.checkNotNull(sparql);
    +        this.childNodeId = childNodeId;
    +        this.graph = graph;
    +        this.sparql = sparql;
    +    }
    +
    +    /**
    +     * @return sparql query string representing this construct query
    +     */
    +    public String getSparql() {
    +        return sparql;
    +    }
    +
    +    /**
    +     * @return The node whose results are projected onto the given
    +     *         {@link ConstructGraph}.
    +     */
    +    public String getChildNodeId() {
    +        return childNodeId;
    +    }
    +
    +    /**
    +     * @return The ConstructGraph used to form statement {@link BindingSet}s for
    +     *         this Construct Query
    +     */
    +    public ConstructGraph getConstructGraph() {
    +        return graph;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(super.getNodeId(), super.getVariableOrder(), childNodeId, graph, sparql);
    +    }
    +
    +    @Override
    +    public boolean equals(final Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +
    +        if (o instanceof ConstructQueryMetadata) {
    +            ConstructQueryMetadata queryMetadata = (ConstructQueryMetadata) o;
    +            if (super.equals(queryMetadata)) {
    +                return new EqualsBuilder().append(childNodeId, queryMetadata.childNodeId).append(graph, queryMetadata.graph)
    +                        .append(sparql, queryMetadata.sparql).isEquals();
    +            }
    +            return false;
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return new StringBuilder().append("Construct Query Metadata {\n").append("    Node ID: " + super.getNodeId() + "\n")
    --- End diff --
    
    no need to implement this, however.


---
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.
---