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/11/22 15:32:28 UTC

[GitHub] incubator-rya pull request #251: Candidate/rya 406

GitHub user meiercaleb opened a pull request:

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

    Candidate/rya 406

    <!--
    Licensed to the Apache Software Foundation (ASF) under one
    or more contributor license agreements.  See the NOTICE file
    distributed with this work for additional information
    regarding copyright ownership.  The ASF licenses this file
    to you under the Apache License, Version 2.0 (the
    "License"); you may not use this file except in compliance
    with the License.  You may obtain a copy of the License at
    
      http://www.apache.org/licenses/LICENSE-2.0
    
    Unless required by applicable law or agreed to in writing,
    software distributed under the License is distributed on an
    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    KIND, either express or implied.  See the License for the
    specific language governing permissions and limitations
    under the License.
    -->
    ## Description
    >What Changed?
    Added optimizations to improve the performance of the Rya Pcj Updater.  These optimizations
    include a metadata cache to store query node metadata, a metadata cache to store statement
    pattern ids, and changes to shard binding set results in the Fluo table.  Changes were made
    to the Rya manual to document these optimizations and provide step by step instructions for how to apply them when deploying an instance of the Pcj Updater.  
    
    ### Tests
    Tests were added for each of the metadata caches and for any utility classes that support
    sharding.  The fluo integration tests provide additional test coverage.
    
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-406)
    
    ### Checklist
    - [ ] Code Review
    - [X] Squash Commits
    
    #### People To Reivew
    @kchilton2 @jdasch @isper3at 


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

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

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

    https://github.com/apache/incubator-rya/pull/251.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 #251
    
----
commit 6d2bfcbcc1fe68e74521724d6f5490a6b9c70038
Author: Caleb Meier <ca...@parsons.com>
Date:   2017-10-26T21:53:56Z

    FluoQueryMetadataCache

commit f7b2cbf40a3ead24a3cf1b74503ac189ffb6a984
Author: Caleb Meier <ca...@parsons.com>
Date:   2017-11-06T20:23:57Z

    RYA-406

----


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153630587
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153007985
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    --- End diff --
    
    document this


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153264717
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    +        if(!commonVars.isEmpty()) {
    +            scanPrefix = getRowKey(siblingId, new VariableOrder(commonVars), childBindingSet);
    +        } else {
    +            scanPrefix = getRowKey(siblingId, siblingVarOrder, childBindingSet);
             }
    -        siblingScanPrefix = siblingId + NODEID_BS_DELIM + siblingScanPrefix;
    -        return Span.prefix(siblingScanPrefix);
    +//
    --- End diff --
    
    Delete commented out code.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153009131
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java ---
    @@ -101,14 +99,14 @@ public void processBatch(TransactionBase tx, Bytes row, BatchInformation batch)
             }
     
             // Insert the new join binding sets to the Fluo table.
    -        final JoinMetadata joinMetadata = dao.readJoinMetadata(tx, nodeId);
    +        final JoinMetadata joinMetadata = CACHE.readJoinMetadata(tx, nodeId);
             final VariableOrder joinVarOrder = joinMetadata.getVariableOrder();
             while (newJoinResults.hasNext()) {
                 final VisibilityBindingSet newJoinResult = newJoinResults.next();
                 //create BindingSet value
                 Bytes bsBytes = BS_SERDE.serialize(newJoinResult);
                 //make rowId
    --- End diff --
    
    doc out of date.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153007574
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/InsertTriples.java ---
    @@ -125,8 +125,8 @@ public void insert(final FluoClient fluo, final Collection<RyaStatement> triples
                 tx.commit();
             }
         }
    -    
    -    
    +
    --- End diff --
    
    nit: clean up this extra whitespace


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153261691
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/IncrementalUpdateConstants.java ---
    @@ -30,15 +30,18 @@
         public static final String TYPE_DELIM = "<<~>>";
     
         //to be used in construction of id for each node
    -    public static final String SP_PREFIX = "STATEMENT_PATTERN";
    -    public static final String JOIN_PREFIX = "JOIN";
    -    public static final String FILTER_PREFIX = "FILTER";
    -    public static final String AGGREGATION_PREFIX = "AGGREGATION";
    -    public static final String QUERY_PREFIX = "QUERY";
    -    public static final String PROJECTION_PREFIX = "PROJECTION";
    -    public static final String CONSTRUCT_PREFIX = "CONSTRUCT";
    -    public static final String PERIODIC_QUERY_PREFIX = "PERIODIC_QUERY";
    -    
    +    public static final String TRIPLE_PREFIX = "T";
    --- End diff --
    
    Was renaming all of these really necessary?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153532275
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153625215
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    --- End diff --
    
    Done.  Gonna hold off on refactoring the suppliers to be the same.  I could maybe do something with generics, but then I'd have to refactor the cache types to extend some common interface.  The cache are different enough that that would be an artificial type def.  Prolly not worth it for the purposes of combining suppliers...Lemme know if you have something different in mind.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153537398
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -70,53 +68,50 @@ public void process(final TransactionBase tx, final Bytes brow, final Column col
             // Get string representation of triple.
             final RyaStatement ryaStatement = IncUpdateDAO.deserializeTriple(brow);
             log.trace("Transaction ID: {}\nRya Statement: {}\n", tx.getStartTimestamp(), ryaStatement);
    +        log.trace("Beginging to process triple.");
     
             final String triple = IncUpdateDAO.getTripleString(ryaStatement);
     
    -        // Iterate over each of the Statement Patterns that are being matched against.
    -        final RowScanner spScanner = tx.scanner()
    -                .over(Span.prefix(SP_PREFIX))
    -
    -                // Only fetch rows that have the pattern in them. There will only be a single row with a pattern per SP.
    -                .fetch(FluoQueryColumns.STATEMENT_PATTERN_PATTERN)
    -                .byRow()
    -                .build();
    +        Set<String> spIDs = SP_ID_CACHE.getStatementPatternIds(tx);
     
             //see if triple matches conditions of any of the SP
    -        for (final ColumnScanner colScanner : spScanner) {
    -            // Get the Statement Pattern's node id.
    -            final String spID = colScanner.getsRow();
    -
    +        for (String spID: spIDs) {
                 // Fetch its metadata.
                 final StatementPatternMetadata spMetadata = QUERY_METADATA_DAO.readStatementPatternMetadata(tx, spID);
     
    +            log.trace("Retrieved metadata: {}", spMetadata);
    +
                 // Attempt to match the triple against the pattern.
                 final String pattern = spMetadata.getStatementPattern();
                 final VariableOrder varOrder = spMetadata.getVariableOrder();
                 final String bindingSetString = getBindingSet(triple, pattern, varOrder);
     
    +            log.trace("Created binding set match string: {}", bindingSetString);
    +
                 // Statement matches to a binding set.
                 if(bindingSetString.length() != 0) {
                     // Fetch the triple's visibility label.
                     final String visibility = tx.gets(brow.toString(), FluoQueryColumns.TRIPLES, "");
     
                     // Create the Row ID for the emitted binding set. It does not contain visibilities.
    -                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    -                final Bytes rowBytes = Bytes.of( row.getBytes(Charsets.UTF_8) );
    +//                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153266074
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
    --- End diff --
    
    Nullness contract.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627442
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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

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


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627110
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153255877
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/MetadataCacheSupplier.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class MetadataCacheSupplier {
    --- End diff --
    
    doc


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008957
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/PeriodicQueryUpdater.java ---
    @@ -65,18 +64,18 @@ public void updatePeriodicBinResults(TransactionBase tx, VisibilityBindingSet bs
                 binnedBs.addBinding(IncrementalUpdateConstants.PERIODIC_BIN_ID, vf.createLiteral(id));
                 VisibilityBindingSet visibilityBindingSet = new VisibilityBindingSet(binnedBs, bs.getVisibility());
                 Bytes periodicBsBytes = BS_SERDE.serialize(visibilityBindingSet);
    -            
    -            //create row 
    -            final Bytes resultRow = RowKeyUtil.makeRowKey(metadata.getNodeId(), metadata.getVariableOrder(), binnedBs);
    +
    +            //create row
    +            final Bytes resultRow = getRowKey(metadata.getNodeId(), metadata.getVariableOrder(), visibilityBindingSet);
    --- End diff --
    
    same create vs get doc here


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263778
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    --- End diff --
    
    null check


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153558938
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(StatementPatternIdCacheSupplier.class);
    +    private static boolean initialized = false;
    +    private static StatementPatternIdCache CACHE;
    +    private static final ReentrantLock lock = new ReentrantLock();
    +
    +    /**
    +     * Returns an existing cache if one has been created, otherwise creates a new cache.
    +     *
    +     * @return - existing StatementPatternIdCache or new cache if one didn't already exist
    +     */
    +    public static StatementPatternIdCache getOrCreateCache() {
    +        lock.lock();
    +        try {
    +            if (!initialized) {
    +                LOG.debug("Cache has not been initialized.  Initializing StatementPatternIdCache");
    +                CACHE = new StatementPatternIdCache();
    +                initialized = true;
    +            } else {
    +                LOG.debug("A StatementPatternIdCache has already been initialized.");
    +            }
    +            return CACHE;
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    /**
    +     * Flushes stored cache and flags Supplier as uninitialized.
    --- End diff --
    
    Changed flushes to deletes in the docs.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153243875
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java ---
    @@ -55,7 +56,7 @@ public Observation parseObservation(final TransactionBase tx, final Bytes row) {
             requireNonNull(row);
     
             // Fetch the Aggregation node's metadata.
    -        final String nodeId = BindingSetRow.make(row).getNodeId();
    +        final String nodeId = BindingSetRow.makeFromShardedRow(Bytes.of(AGGREGATION_PREFIX), row).getNodeId();
    --- End diff --
    
    docs out of date


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153629802
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153608050
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCache.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.collect.Sets;
    +
    +/**
    + * This class caches the StatementPattern Ids so they don't have
    + * to be looked up each time a new Statement needs to be processed
    + * in the TripleObserver.
    + *
    + */
    +public class StatementPatternIdCache {
    +
    +    private final ReentrantLock lock = new ReentrantLock();
    +    private static Optional<String> HASH;
    +    private static Set<String> IDS;
    +
    +    public StatementPatternIdCache() {
    +        HASH = Optional.empty();
    +        IDS = new HashSet<>();
    +    }
    +
    +    /**
    +     * This method retrieves the StatementPattern NodeIds registered in the Fluo table.
    +     * This method looks up the hash of the Statement Pattern Id String hash, and if it
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153578897
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -70,53 +68,50 @@ public void process(final TransactionBase tx, final Bytes brow, final Column col
             // Get string representation of triple.
             final RyaStatement ryaStatement = IncUpdateDAO.deserializeTriple(brow);
             log.trace("Transaction ID: {}\nRya Statement: {}\n", tx.getStartTimestamp(), ryaStatement);
    +        log.trace("Beginging to process triple.");
     
             final String triple = IncUpdateDAO.getTripleString(ryaStatement);
     
    -        // Iterate over each of the Statement Patterns that are being matched against.
    -        final RowScanner spScanner = tx.scanner()
    -                .over(Span.prefix(SP_PREFIX))
    -
    -                // Only fetch rows that have the pattern in them. There will only be a single row with a pattern per SP.
    -                .fetch(FluoQueryColumns.STATEMENT_PATTERN_PATTERN)
    -                .byRow()
    -                .build();
    +        Set<String> spIDs = SP_ID_CACHE.getStatementPatternIds(tx);
     
             //see if triple matches conditions of any of the SP
    -        for (final ColumnScanner colScanner : spScanner) {
    -            // Get the Statement Pattern's node id.
    -            final String spID = colScanner.getsRow();
    -
    +        for (String spID: spIDs) {
                 // Fetch its metadata.
                 final StatementPatternMetadata spMetadata = QUERY_METADATA_DAO.readStatementPatternMetadata(tx, spID);
     
    +            log.trace("Retrieved metadata: {}", spMetadata);
    +
                 // Attempt to match the triple against the pattern.
                 final String pattern = spMetadata.getStatementPattern();
                 final VariableOrder varOrder = spMetadata.getVariableOrder();
                 final String bindingSetString = getBindingSet(triple, pattern, varOrder);
     
    +            log.trace("Created binding set match string: {}", bindingSetString);
    +
                 // Statement matches to a binding set.
                 if(bindingSetString.length() != 0) {
                     // Fetch the triple's visibility label.
                     final String visibility = tx.gets(brow.toString(), FluoQueryColumns.TRIPLES, "");
     
                     // Create the Row ID for the emitted binding set. It does not contain visibilities.
    -                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    -                final Bytes rowBytes = Bytes.of( row.getBytes(Charsets.UTF_8) );
    +//                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153588548
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (ConstructQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readConstructQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ConstructQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public PeriodicQueryMetadata readPeriodicQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.PERIODIC_QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (PeriodicQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readPeriodicQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access PeriodicQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public QueryMetadata readQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (QueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access QueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    public Bytes readMetadadataEntry(SnapshotBase tx, String rowId, Column column) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153582032
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    --- End diff --
    
    The delete question might actually be a problem.  I'm going to create a Jira ticket for this.  We need some way to notify the workers when metadata has been deleted so that the caches on each JVM can purge the metadata for any deleted nodeIds.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533846
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/QueryResultUpdater.java ---
    @@ -70,7 +69,7 @@ public void updateQueryResults(
             final VariableOrder queryVarOrder = queryMetadata.getVariableOrder();
     
             // Create the Row Key for the result. If the child node groups results, then the key must only contain the Group By variables.
    -        final Bytes resultRow = RowKeyUtil.makeRowKey(queryMetadata.getNodeId(), queryVarOrder, childBindingSet);
    +        final Bytes resultRow = getRowKey(queryMetadata.getNodeId(), queryVarOrder, childBindingSet);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153630157
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    +        BytesBuilder builder = Bytes.builder();
    +        return builder.append(TRIPLE_PREFIX_BYTES).append(tripleBytes).toBytes();
    +    }
    +
    +    /**
    +     * Removes the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to a byte array.
    +     * @param prefixedTriple - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     * @return - serialized {@link RyaStatement} in byte array form
    +     */
    +    public static byte[] removeTriplePrefixAndConvertToByteArray(Bytes prefixedTriple) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153262219
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(StatementPatternIdCacheSupplier.class);
    +    private static boolean initialized = false;
    +    private static StatementPatternIdCache CACHE;
    +    private static final ReentrantLock lock = new ReentrantLock();
    +
    +    /**
    +     * Returns an existing cache if one has been created, otherwise creates a new cache.
    +     *
    +     * @return - existing StatementPatternIdCache or new cache if one didn't already exist
    +     */
    +    public static StatementPatternIdCache getOrCreateCache() {
    +        lock.lock();
    +        try {
    +            if (!initialized) {
    +                LOG.debug("Cache has not been initialized.  Initializing StatementPatternIdCache");
    +                CACHE = new StatementPatternIdCache();
    +                initialized = true;
    +            } else {
    +                LOG.debug("A StatementPatternIdCache has already been initialized.");
    +            }
    +            return CACHE;
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    /**
    +     * Flushes stored cache and flags Supplier as uninitialized.
    --- End diff --
    
    should this be renamed to flush() then?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153254552
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    --- End diff --
    
    catch checkArg()


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153556712
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    --- End diff --
    
    Same.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153249238
  
    --- Diff: extras/indexingExample/src/main/java/RyaClientExample.java ---
    @@ -75,7 +76,7 @@
         private static final Logger log = Logger.getLogger(RyaClientExample.class);
     
         public static void main(final String[] args) throws Exception {
    -        setupLogging();
    +//        setupLogging();
    --- End diff --
    
    Should this be commented out? If it's not being used, we should remove it.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r152997346
  
    --- Diff: extras/indexingExample/src/main/java/RyaClientExample.java ---
    @@ -75,7 +76,7 @@
         private static final Logger log = Logger.getLogger(RyaClientExample.class);
     
         public static void main(final String[] args) throws Exception {
    -        setupLogging();
    +//        setupLogging();
    --- End diff --
    
    no more logging?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153248632
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -70,53 +68,50 @@ public void process(final TransactionBase tx, final Bytes brow, final Column col
             // Get string representation of triple.
             final RyaStatement ryaStatement = IncUpdateDAO.deserializeTriple(brow);
             log.trace("Transaction ID: {}\nRya Statement: {}\n", tx.getStartTimestamp(), ryaStatement);
    +        log.trace("Beginging to process triple.");
     
             final String triple = IncUpdateDAO.getTripleString(ryaStatement);
     
    -        // Iterate over each of the Statement Patterns that are being matched against.
    -        final RowScanner spScanner = tx.scanner()
    -                .over(Span.prefix(SP_PREFIX))
    -
    -                // Only fetch rows that have the pattern in them. There will only be a single row with a pattern per SP.
    -                .fetch(FluoQueryColumns.STATEMENT_PATTERN_PATTERN)
    -                .byRow()
    -                .build();
    +        Set<String> spIDs = SP_ID_CACHE.getStatementPatternIds(tx);
     
             //see if triple matches conditions of any of the SP
    -        for (final ColumnScanner colScanner : spScanner) {
    -            // Get the Statement Pattern's node id.
    -            final String spID = colScanner.getsRow();
    -
    +        for (String spID: spIDs) {
                 // Fetch its metadata.
                 final StatementPatternMetadata spMetadata = QUERY_METADATA_DAO.readStatementPatternMetadata(tx, spID);
     
    +            log.trace("Retrieved metadata: {}", spMetadata);
    +
                 // Attempt to match the triple against the pattern.
                 final String pattern = spMetadata.getStatementPattern();
                 final VariableOrder varOrder = spMetadata.getVariableOrder();
                 final String bindingSetString = getBindingSet(triple, pattern, varOrder);
     
    +            log.trace("Created binding set match string: {}", bindingSetString);
    +
                 // Statement matches to a binding set.
                 if(bindingSetString.length() != 0) {
                     // Fetch the triple's visibility label.
                     final String visibility = tx.gets(brow.toString(), FluoQueryColumns.TRIPLES, "");
     
                     // Create the Row ID for the emitted binding set. It does not contain visibilities.
    -                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    -                final Bytes rowBytes = Bytes.of( row.getBytes(Charsets.UTF_8) );
    +//                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    --- End diff --
    
    commented out code


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153253825
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/pom.xml ---
    @@ -1,25 +1,16 @@
     <?xml version="1.0" encoding="utf-8"?>
    -<!--
    -
    -    Licensed to the Apache Software Foundation (ASF) under one
    -    or more contributor license agreements.  See the NOTICE file
    -    distributed with this work for additional information
    -    regarding copyright ownership.  The ASF licenses this file
    -    to you under the Apache License, Version 2.0 (the
    -    "License"); you may not use this file except in compliance
    -    with the License.  You may obtain a copy of the License at
    -
    -        http://www.apache.org/licenses/LICENSE-2.0
    -
    -    Unless required by applicable law or agreed to in writing,
    -    software distributed under the License is distributed on an
    -    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    -    KIND, either express or implied.  See the License for the
    -    specific language governing permissions and limitations
    -    under the License.
    -
    --->
    -<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
    --- End diff --
    
    Please leave the license how it was originally formatted.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153254708
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AbstractNodeUpdater.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public abstract class AbstractNodeUpdater {
    --- End diff --
    
    It also doesn't necessarily make sense to be an abstract class since there are no abstract methods on it. Could the public function just be a utility somewhere? Is there other logic that could be shared with its children?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153262388
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    --- End diff --
    
    Should this use IncrementalUpdateConstants.TRIPLE_PREFIX?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153528177
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -104,7 +107,7 @@ public void deletePcj(final FluoClient client, final String pcjId) throws Unsupp
          * @param tx - Transaction of a given Fluo table. (not null)
          * @param pcjId - Id of query. (not null)
          * @return list of Node IDs associated with the query {@code pcjId}.
    -     * @throws UnsupportedQueryException 
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627036
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533498
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153589892
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCache.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.collect.Sets;
    +
    +/**
    + * This class caches the StatementPattern Ids so they don't have
    + * to be looked up each time a new Statement needs to be processed
    + * in the TripleObserver.
    + *
    + */
    +public class StatementPatternIdCache {
    --- End diff --
    
    Not including that.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153588977
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/MetadataCacheSupplier.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class MetadataCacheSupplier {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153570866
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    --- End diff --
    
    See above.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274129
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    --- End diff --
    
    Could you talk about the eviction strategy up here? Does it only evict based on capacity? What if a query was deleted? How long will it take for that to be evicted from the cache? Might we compute results for deleted queries in the mean time?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153252634
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    --- End diff --
    
    won't this catch(Exception) catch a checkArgument() failure?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008987
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ProjectionResultUpdater.java ---
    @@ -65,20 +64,21 @@ public void updateProjectionResults(
             log.trace(
                     "Transaction ID: " + tx.getStartTimestamp() + "\n" +
                     "Node ID: " + projectionMetadata.getNodeId() + "\n" +
    -                "Parent Node ID: " + projectionMetadata.getParentNodeId() + "\n" +        
    +                "Parent Node ID: " + projectionMetadata.getParentNodeId() + "\n" +
                     "Child Node ID: " + projectionMetadata.getChildNodeId() + "\n" +
                     "Child Binding Set:\n" + childBindingSet + "\n");
     
             // Create the query's Binding Set from the child node's binding set.
             final VariableOrder queryVarOrder = projectionMetadata.getVariableOrder();
             final VariableOrder projectionVarOrder = projectionMetadata.getProjectedVars();
             final BindingSet queryBindingSet = BindingSetUtil.keepBindings(projectionVarOrder, childBindingSet);
    +        VisibilityBindingSet projectedBs = new VisibilityBindingSet(queryBindingSet, childBindingSet.getVisibility());
     
             // Create the Row Key for the result. If the child node groups results, then the key must only contain the Group By variables.
    -        Bytes resultRow  = RowKeyUtil.makeRowKey(projectionMetadata.getNodeId(), queryVarOrder, queryBindingSet);
    +        Bytes resultRow  = getRowKey(projectionMetadata.getNodeId(), queryVarOrder, projectedBs);
    --- End diff --
    
    create vs get


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153631182
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    +        Preconditions.checkArgument(rowPrefixAndId.length == 2);
    +        String prefix = rowPrefixAndId[0];
    +        String id = rowPrefixAndId[1];
    +
    +        String firstBindingString = "";
    +        Bytes rowSuffix = Bytes.of(id);
    +        if (varOrder.getVariableOrders().size() > 0) {
    +            VariableOrder first = new VariableOrder(varOrder.getVariableOrders().get(0));
    +            firstBindingString = BS_CONVERTER.convert(bs, first);
    +            rowSuffix = RowKeyUtil.makeRowKey(id, varOrder, bs);
    +        }
    +
    +        BytesBuilder builder = Bytes.builder();
    +        builder.append(Bytes.of(prefix + ":"));
    +        builder.append(genHash(Bytes.of(id + NODEID_BS_DELIM + firstBindingString)));
    +        builder.append(":");
    +        builder.append(rowSuffix);
    +        return builder.toBytes();
    +    }
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param firstBsVal - String representation of the first BsValue
    +     * @return - serialized Bytes prefix for scanning rows
    +     */
    +    public static Bytes getShardedScanPrefix(String nodeId, Value firstBsVal) {
    +        Preconditions.checkNotNull(firstBsVal);
    +
    +        final RyaType ryaValue = RdfToRyaConversions.convertValue(firstBsVal);
    +        final String bindingString = ryaValue.getData() + TYPE_DELIM + ryaValue.getDataType();
    +
    +        return getShardedScanPrefix(nodeId, bindingString);
    +    }
    +
    +    /**
    +     * Generates a sharded rowId from the indicated nodeId and bindingString.
    +     *
    +     * @param nodeId - NodeId with tyep and UUID
    +     * @param bindingString - String representation of BindingSet values, as formed by {@link BindingSetStringConverter}
    +     *            .
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153583702
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    --- End diff --
    
    Yeah, not doing that.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153585160
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153532849
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -145,8 +139,8 @@ public void updateJoinResults(
                 final VisibilityBindingSet newJoinResult = newJoinResults.next();
     
                 // Create the Row Key for the emitted binding set. It does not contain visibilities.
    -            final Bytes resultRow = RowKeyUtil.makeRowKey(joinMetadata.getNodeId(), joinVarOrder, newJoinResult);
    -            
    +            final Bytes resultRow = getRowKey(joinMetadata.getNodeId(), joinVarOrder, newJoinResult);
    --- End diff --
    
    Changed back to make.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153250295
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -104,7 +107,7 @@ public void deletePcj(final FluoClient client, final String pcjId) throws Unsupp
          * @param tx - Transaction of a given Fluo table. (not null)
          * @param pcjId - Id of query. (not null)
          * @return list of Node IDs associated with the query {@code pcjId}.
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    Could you also add the documentation here?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153266257
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    --- End diff --
    
    What hash? Does it have a special name?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153625655
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    --- End diff --
    
    Yeah, not going to add that.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153557967
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/MetadataCacheSupplier.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class MetadataCacheSupplier {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153588803
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (ConstructQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readConstructQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ConstructQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public PeriodicQueryMetadata readPeriodicQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.PERIODIC_QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (PeriodicQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readPeriodicQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access PeriodicQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public QueryMetadata readQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (QueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access QueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    public Bytes readMetadadataEntry(SnapshotBase tx, String rowId, Column column) {
    +        Optional<NodeType> type = NodeType.fromNodeId(rowId);
    +        try {
    +            checkArgument(type.isPresent() && type.get().getMetaDataColumns().contains(column));
    +            return metadataCache.get(getKey(rowId, column), new Callable<Bytes>() {
    +                @Override
    +                public Bytes call() throws Exception {
    +                    return tx.get(Bytes.of(rowId), column);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access Metadata Entry with rowId: " + rowId + " and column: " + column, e);
    +        }
    +    }
    +
    +    public void clear() {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153539036
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    --- End diff --
    
    Yeah, but it gets repackaged as a RuntimeException.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153535855
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java ---
    @@ -55,7 +56,7 @@ public Observation parseObservation(final TransactionBase tx, final Bytes row) {
             requireNonNull(row);
     
             // Fetch the Aggregation node's metadata.
    -        final String nodeId = BindingSetRow.make(row).getNodeId();
    +        final String nodeId = BindingSetRow.makeFromShardedRow(Bytes.of(AGGREGATION_PREFIX), row).getNodeId();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153559862
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to remove from the StatementPattern nodeId Set
    +     */
    +    public static void removeStatementPatternIds(TransactionBase tx, Set<String> ids) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153562935
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/GetQueryReport.java ---
    @@ -63,7 +66,7 @@
          * @param fluo - The connection to Fluo that will be used to fetch the metadata. (not null)
          * @return A map from Query ID to QueryReport that holds a report for all of
          *   the queries that are being managed within the fluo app.
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    This preexists my commits.  Leaving as is.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153624484
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCache.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.collect.Sets;
    +
    +/**
    + * This class caches the StatementPattern Ids so they don't have
    + * to be looked up each time a new Statement needs to be processed
    + * in the TripleObserver.
    + *
    + */
    +public class StatementPatternIdCache {
    +
    +    private final ReentrantLock lock = new ReentrantLock();
    +    private static Optional<String> HASH;
    +    private static Set<String> IDS;
    +
    +    public StatementPatternIdCache() {
    +        HASH = Optional.empty();
    +        IDS = new HashSet<>();
    +    }
    +
    +    /**
    +     * This method retrieves the StatementPattern NodeIds registered in the Fluo table.
    +     * This method looks up the hash of the Statement Pattern Id String hash, and if it
    +     * is the same as the cached hash, then the cache Set of nodeIds is returned.  Otherwise,
    +     * this method retrieves the ids from the Fluo table.  This method is thread safe.
    +     * @param tx
    +     * @return - Set of StatementPattern nodeIds
    +     */
    +    public Set<String> getStatementPatternIds(TransactionBase tx) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153573207
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    +        if(other == null) { return false;}
    --- End diff --
    
    Yep. Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153583867
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    --- End diff --
    
    I like that.  I'll keep it in mind going forward.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008783
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    --- End diff --
    
    remove commented code


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263311
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    +        BytesBuilder builder = Bytes.builder();
    +        return builder.append(TRIPLE_PREFIX_BYTES).append(tripleBytes).toBytes();
    +    }
    +
    +    /**
    +     * Removes the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to a byte array.
    --- End diff --
    
    Same comment.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153267721
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -21,15 +21,16 @@
     import static com.google.common.base.Preconditions.checkNotNull;
     import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
     
    +import org.apache.commons.lang.builder.EqualsBuilder;
     import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
     
     import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
     import edu.umd.cs.findbugs.annotations.NonNull;
     import net.jcip.annotations.Immutable;
     
     /**
    - * The values of an Accumulo Row ID for a row that stores a Binding set for
    - * a specific Node ID of a query.
    + * The values of an Accumulo Row ID for a row that stores a Binding set for a specific Node ID of a query.
    --- End diff --
    
    I think this class needs to explain how exactly these rows are constructed/used since it seems like sometimes they contain hashes, but it's unclear how they are structured.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153254955
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AbstractNodeUpdater.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public abstract class AbstractNodeUpdater {
    --- End diff --
    
    I see the logic used to be in "RowKeyUtil.makeRowKey". You might want to just revert this one out.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627243
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153275927
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (ConstructQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readConstructQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ConstructQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public PeriodicQueryMetadata readPeriodicQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.PERIODIC_QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (PeriodicQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readPeriodicQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access PeriodicQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public QueryMetadata readQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.QUERY);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (QueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access QueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    public Bytes readMetadadataEntry(SnapshotBase tx, String rowId, Column column) {
    --- End diff --
    
    Documentation.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153584424
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153265278
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    --- End diff --
    
    Maybe still call this siblingScanPrefix.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153560956
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153575367
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153629832
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    +        BytesBuilder builder = Bytes.builder();
    +        return builder.append(TRIPLE_PREFIX_BYTES).append(tripleBytes).toBytes();
    +    }
    +
    +    /**
    +     * Removes the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to a byte array.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153273958
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    --- End diff --
    
    "capacity" and "concurrencyLevel" need to be documented.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533789
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/ProjectionResultUpdater.java ---
    @@ -65,20 +64,21 @@ public void updateProjectionResults(
             log.trace(
                     "Transaction ID: " + tx.getStartTimestamp() + "\n" +
                     "Node ID: " + projectionMetadata.getNodeId() + "\n" +
    -                "Parent Node ID: " + projectionMetadata.getParentNodeId() + "\n" +        
    +                "Parent Node ID: " + projectionMetadata.getParentNodeId() + "\n" +
                     "Child Node ID: " + projectionMetadata.getChildNodeId() + "\n" +
                     "Child Binding Set:\n" + childBindingSet + "\n");
     
             // Create the query's Binding Set from the child node's binding set.
             final VariableOrder queryVarOrder = projectionMetadata.getVariableOrder();
             final VariableOrder projectionVarOrder = projectionMetadata.getProjectedVars();
             final BindingSet queryBindingSet = BindingSetUtil.keepBindings(projectionVarOrder, childBindingSet);
    +        VisibilityBindingSet projectedBs = new VisibilityBindingSet(queryBindingSet, childBindingSet.getVisibility());
     
             // Create the Row Key for the result. If the child node groups results, then the key must only contain the Group By variables.
    -        Bytes resultRow  = RowKeyUtil.makeRowKey(projectionMetadata.getNodeId(), queryVarOrder, queryBindingSet);
    +        Bytes resultRow  = getRowKey(projectionMetadata.getNodeId(), queryVarOrder, projectedBs);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153556772
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    --- End diff --
    
    Same.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153557062
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (ConstructQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readConstructQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ConstructQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public PeriodicQueryMetadata readPeriodicQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.PERIODIC_QUERY);
    --- End diff --
    
    Same as above.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533584
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    +        if(!commonVars.isEmpty()) {
    +            scanPrefix = getRowKey(siblingId, new VariableOrder(commonVars), childBindingSet);
    +        } else {
    +            scanPrefix = getRowKey(siblingId, siblingVarOrder, childBindingSet);
             }
    -        siblingScanPrefix = siblingId + NODEID_BS_DELIM + siblingScanPrefix;
    -        return Span.prefix(siblingScanPrefix);
    +//
    +//        // Create the prefix that will be used to scan for binding sets of the sibling node.
    +//        // This prefix includes the sibling Node ID and the common variable values from
    +//        // childBindingSet.
    +//        String siblingScanPrefix = "";
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627802
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    --- End diff --
    
    Nuuupe.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153265966
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    +     * is not included in the Span.
    +     * @param batchSize - size of batch to be deleted
    +     * @param column - column whose entries will be deleted
    +     * @param span - Span indicating the range of data to delete.  Sometimes the Span cannot contain the hash
    +     * (for example, if you are deleting all of the results associated with a nodeId).  In this case, a nodeId
    +     * should be specified along with a Span equal to the prefix of the nodeId.
    +     */
    +    public SpanBatchDeleteInformation(Optional<String> nodeId, int batchSize, Column column, Span span) {
             super(batchSize, Task.Delete, column, span);
    +        Preconditions.checkNotNull(nodeId);
    --- End diff --
    
    requireNonNull


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533667
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/PeriodicQueryUpdater.java ---
    @@ -65,18 +64,18 @@ public void updatePeriodicBinResults(TransactionBase tx, VisibilityBindingSet bs
                 binnedBs.addBinding(IncrementalUpdateConstants.PERIODIC_BIN_ID, vf.createLiteral(id));
                 VisibilityBindingSet visibilityBindingSet = new VisibilityBindingSet(binnedBs, bs.getVisibility());
                 Bytes periodicBsBytes = BS_SERDE.serialize(visibilityBindingSet);
    -            
    -            //create row 
    -            final Bytes resultRow = RowKeyUtil.makeRowKey(metadata.getNodeId(), metadata.getVariableOrder(), binnedBs);
    +
    +            //create row
    +            final Bytes resultRow = getRowKey(metadata.getNodeId(), metadata.getVariableOrder(), visibilityBindingSet);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya issue #251: Candidate/rya 406

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

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



---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153561179
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    +        Preconditions.checkArgument(rowPrefixAndId.length == 2);
    +        String prefix = rowPrefixAndId[0];
    +        String id = rowPrefixAndId[1];
    +
    +        String firstBindingString = "";
    +        Bytes rowSuffix = Bytes.of(id);
    +        if (varOrder.getVariableOrders().size() > 0) {
    +            VariableOrder first = new VariableOrder(varOrder.getVariableOrders().get(0));
    +            firstBindingString = BS_CONVERTER.convert(bs, first);
    +            rowSuffix = RowKeyUtil.makeRowKey(id, varOrder, bs);
    +        }
    +
    +        BytesBuilder builder = Bytes.builder();
    +        builder.append(Bytes.of(prefix + ":"));
    +        builder.append(genHash(Bytes.of(id + NODEID_BS_DELIM + firstBindingString)));
    +        builder.append(":");
    +        builder.append(rowSuffix);
    +        return builder.toBytes();
    +    }
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param firstBsVal - String representation of the first BsValue
    +     * @return - serialized Bytes prefix for scanning rows
    +     */
    +    public static Bytes getShardedScanPrefix(String nodeId, Value firstBsVal) {
    +        Preconditions.checkNotNull(firstBsVal);
    +
    +        final RyaType ryaValue = RdfToRyaConversions.convertValue(firstBsVal);
    +        final String bindingString = ryaValue.getData() + TYPE_DELIM + ryaValue.getDataType();
    +
    +        return getShardedScanPrefix(nodeId, bindingString);
    +    }
    +
    +    /**
    +     * Generates a sharded rowId from the indicated nodeId and bindingString.
    +     *
    +     * @param nodeId - NodeId with tyep and UUID
    +     * @param bindingString - String representation of BindingSet values, as formed by {@link BindingSetStringConverter}
    +     *            .
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153589466
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/MetadataCacheSupplier.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class MetadataCacheSupplier {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(MetadataCacheSupplier.class);
    +    private static FluoQueryMetadataCache CACHE;
    +    private static boolean initialized = false;
    +    private static final int DEFAULT_CAPACITY = 10000;
    +    private static final int DEFAULT_CONCURRENCY = 8;
    +    private static final ReentrantLock lock = new ReentrantLock();
    +
    +    /**
    +     * Returns an existing cache with the specified instance name, or creates a cache. The created cache will have the
    --- End diff --
    
    It does.  It enforces "singleton like" behavior in that it only creates the instance if it doesn't already exist.  It allows for the initial cache construciton to be configurable.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263364
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    --- End diff --
    
    adds or removes?  if it does both....


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274823
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    --- End diff --
    
    Nullness contract.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153574758
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/IncrementalUpdateConstants.java ---
    @@ -30,15 +30,18 @@
         public static final String TYPE_DELIM = "<<~>>";
     
         //to be used in construction of id for each node
    -    public static final String SP_PREFIX = "STATEMENT_PATTERN";
    -    public static final String JOIN_PREFIX = "JOIN";
    -    public static final String FILTER_PREFIX = "FILTER";
    -    public static final String AGGREGATION_PREFIX = "AGGREGATION";
    -    public static final String QUERY_PREFIX = "QUERY";
    -    public static final String PROJECTION_PREFIX = "PROJECTION";
    -    public static final String CONSTRUCT_PREFIX = "CONSTRUCT";
    -    public static final String PERIODIC_QUERY_PREFIX = "PERIODIC_QUERY";
    -    
    +    public static final String TRIPLE_PREFIX = "T";
    --- End diff --
    
    No.  Just more efficient.  Since I was making so many changes to how results were stored, I figured it was extremely unnecessary to have such large prefixes.  Also, when specifying splits in the fluo app, it is necessary to indicate the prefixes that you want to split over.  This becomes cumbersome with such large prefixes.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153247576
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/StatementPatternObserver.java ---
    @@ -49,7 +50,7 @@ public Observation parseObservation(final TransactionBase tx, final Bytes row) t
             requireNonNull(row);
     
             // Read the Statement Pattern metadata.
    -        final String spNodeId = BindingSetRow.make(row).getNodeId();
    +        final String spNodeId = BindingSetRow.makeFromShardedRow(Bytes.of(SP_PREFIX), row).getNodeId();
    --- End diff --
    
    read?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153006896
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -104,7 +107,7 @@ public void deletePcj(final FluoClient client, final String pcjId) throws Unsupp
          * @param tx - Transaction of a given Fluo table. (not null)
          * @param pcjId - Id of query. (not null)
          * @return list of Node IDs associated with the query {@code pcjId}.
    -     * @throws UnsupportedQueryException 
    --- End diff --
    
    can you add a quick description to these for when this will be thrown?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153531369
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/InsertTriples.java ---
    @@ -125,8 +125,8 @@ public void insert(final FluoClient fluo, final Collection<RyaStatement> triples
                 tx.commit();
             }
         }
    -    
    -    
    +
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153527315
  
    --- Diff: extras/indexingExample/src/main/java/RyaClientExample.java ---
    @@ -75,7 +76,7 @@
         private static final Logger log = Logger.getLogger(RyaClientExample.class);
     
         public static void main(final String[] args) throws Exception {
    -        setupLogging();
    +//        setupLogging();
    --- End diff --
    
    This method was generating a NPE for some reason.  There were changes made to our logging framework that might be the culprit.  Test passes without it and there is logging.  Not going down that rabbit hole.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153252727
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    --- End diff --
    
    same


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153630697
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to remove from the StatementPattern nodeId Set
    +     */
    +    public static void removeStatementPatternIds(TransactionBase tx, Set<String> ids) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153631009
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    --- End diff --
    
    It does both.  Reworded to adds or removes...


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008694
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -145,8 +139,8 @@ public void updateJoinResults(
                 final VisibilityBindingSet newJoinResult = newJoinResults.next();
     
                 // Create the Row Key for the emitted binding set. It does not contain visibilities.
    -            final Bytes resultRow = RowKeyUtil.makeRowKey(joinMetadata.getNodeId(), joinVarOrder, newJoinResult);
    -            
    +            final Bytes resultRow = getRowKey(joinMetadata.getNodeId(), joinVarOrder, newJoinResult);
    --- End diff --
    
    rewording makeRowKey -> getRowKey makes the docs a little out of date.  Doesn't seem like you're creating a row key here


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263046
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to remove from the StatementPattern nodeId Set
    +     */
    +    public static void removeStatementPatternIds(TransactionBase tx, Set<String> ids) {
    --- End diff --
    
    null checks on the params


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153264693
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    --- End diff --
    
    Delete commented out code.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153275010
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    --- End diff --
    
    This comment applies for all the methods.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153252995
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/GetQueryReport.java ---
    @@ -63,7 +66,7 @@
          * @param fluo - The connection to Fluo that will be used to fetch the metadata. (not null)
          * @return A map from Query ID to QueryReport that holds a report for all of
          *   the queries that are being managed within the fluo app.
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    Same comment about documenting these exceptions through out this class.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153578792
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    +     * is not included in the Span.
    +     * @param batchSize - size of batch to be deleted
    +     * @param column - column whose entries will be deleted
    +     * @param span - Span indicating the range of data to delete.  Sometimes the Span cannot contain the hash
    +     * (for example, if you are deleting all of the results associated with a nodeId).  In this case, a nodeId
    +     * should be specified along with a Span equal to the prefix of the nodeId.
    +     */
    +    public SpanBatchDeleteInformation(Optional<String> nodeId, int batchSize, Column column, Span span) {
             super(batchSize, Task.Delete, column, span);
    +        Preconditions.checkNotNull(nodeId);
    --- End diff --
    
    checkNotNull and requireNonNull are intermixed everywhere.  Not worried about this.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153533203
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -183,18 +177,18 @@ public void updateJoinResults(
         public static enum Side {
             LEFT, RIGHT;
         }
    -    
    -    
    +
    +
         /**
          * Fetches batch to be processed by scanning over the Span specified by the
          * {@link JoinBatchInformation}. The number of results is less than or equal
          * to the batch size specified by the JoinBatchInformation.
    -     * 
    +     *
          * @param tx - Fluo transaction in which batch operation is performed
          * @param siblingSpan - span of sibling to retrieve elements to join with
          * @param bsSet- set that batch results are added to
          * @return Set - containing results of sibling scan.
    -     * @throws Exception 
    +     * @throws Exception
    --- End diff --
    
    Not my change.  Leaving as is.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153247668
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -55,11 +54,10 @@
         private static final Logger log = LoggerFactory.getLogger(TripleObserver.class);
     
         private static final VisibilityBindingSetSerDe BS_SERDE = new VisibilityBindingSetSerDe();
    -    private static final FluoQueryMetadataDAO QUERY_METADATA_DAO = new FluoQueryMetadataDAO();
    +    private final FluoQueryMetadataCache QUERY_METADATA_DAO = MetadataCacheSupplier.getOrCreateCache();;
    --- End diff --
    
    double semicolons


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274878
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    --- End diff --
    
    null check


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153631103
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153559519
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153572801
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153257508
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    +        if(other == null) { return false;}
    --- End diff --
    
    You can remove this line since "other instanceof BindingSetRow" will return false when other is null.


---

[GitHub] incubator-rya issue #251: Candidate/rya 406

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

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



---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008833
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    +        if(!commonVars.isEmpty()) {
    +            scanPrefix = getRowKey(siblingId, new VariableOrder(commonVars), childBindingSet);
    +        } else {
    +            scanPrefix = getRowKey(siblingId, siblingVarOrder, childBindingSet);
             }
    -        siblingScanPrefix = siblingId + NODEID_BS_DELIM + siblingScanPrefix;
    -        return Span.prefix(siblingScanPrefix);
    +//
    +//        // Create the prefix that will be used to scan for binding sets of the sibling node.
    +//        // This prefix includes the sibling Node ID and the common variable values from
    +//        // childBindingSet.
    +//        String siblingScanPrefix = "";
    --- End diff --
    
    more commented out code


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153627306
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    --- End diff --
    
    Not doing this.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153587213
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    --- End diff --
    
    Ooh.  That is elegant.  I'll keep it in mind going forward.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153282346
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    +        Preconditions.checkArgument(rowPrefixAndId.length == 2);
    +        String prefix = rowPrefixAndId[0];
    +        String id = rowPrefixAndId[1];
    +
    +        String firstBindingString = "";
    +        Bytes rowSuffix = Bytes.of(id);
    +        if (varOrder.getVariableOrders().size() > 0) {
    +            VariableOrder first = new VariableOrder(varOrder.getVariableOrders().get(0));
    +            firstBindingString = BS_CONVERTER.convert(bs, first);
    +            rowSuffix = RowKeyUtil.makeRowKey(id, varOrder, bs);
    +        }
    +
    +        BytesBuilder builder = Bytes.builder();
    +        builder.append(Bytes.of(prefix + ":"));
    +        builder.append(genHash(Bytes.of(id + NODEID_BS_DELIM + firstBindingString)));
    +        builder.append(":");
    +        builder.append(rowSuffix);
    +        return builder.toBytes();
    +    }
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param firstBsVal - String representation of the first BsValue
    +     * @return - serialized Bytes prefix for scanning rows
    +     */
    +    public static Bytes getShardedScanPrefix(String nodeId, Value firstBsVal) {
    --- End diff --
    
    null check nodeId


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153556972
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    --- End diff --
    
    Repackaged as Runtime Exception.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153558448
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153250145
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -79,7 +82,7 @@ public DeleteFluoPcj(final int batchSize) {
          *            Index. (not null)
          * @param pcjId - The PCJ ID for the query that will removed from the Fluo
          *            application. (not null)
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    I know this isn't new, but could you document when this is thrown?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274717
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    --- End diff --
    
    Add a message for the exception that will be thrown by this method.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153565747
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AbstractNodeUpdater.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public abstract class AbstractNodeUpdater {
    --- End diff --
    
    The class provides the common functionality for generating row keys to all of the updaters that extend it.  By shard row creation in the parent, it limits the number of files that I would have to update to change the strategy for generating row keys.  I think it's fine as is.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153530359
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/InsertTriples.java ---
    @@ -116,7 +116,7 @@ public void insert(final FluoClient fluo, final Collection<RyaStatement> triples
                 for(final RyaStatement triple : triples) {
                     Optional<byte[]> visibility = Optional.fromNullable(triple.getColumnVisibility());
                     try {
    -                    tx.set(Bytes.of(spoFormat(triple)), FluoQueryColumns.TRIPLES, Bytes.of(visibility.or(new byte[0])));
    +                    tx.set(spoFormat(triple), FluoQueryColumns.TRIPLES, Bytes.of(visibility.or(new byte[0])));
    --- End diff --
    
    That wasn't my addition.  Leaving as is.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153628506
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153536190
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/StatementPatternObserver.java ---
    @@ -49,7 +50,7 @@ public Observation parseObservation(final TransactionBase tx, final Bytes row) t
             requireNonNull(row);
     
             // Read the Statement Pattern metadata.
    -        final String spNodeId = BindingSetRow.make(row).getNodeId();
    +        final String spNodeId = BindingSetRow.makeFromShardedRow(Bytes.of(SP_PREFIX), row).getNodeId();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263703
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    +        BytesBuilder builder = Bytes.builder();
    +        return builder.append(TRIPLE_PREFIX_BYTES).append(tripleBytes).toBytes();
    +    }
    +
    +    /**
    +     * Removes the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to a byte array.
    +     * @param prefixedTriple - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     * @return - serialized {@link RyaStatement} in byte array form
    +     */
    +    public static byte[] removeTriplePrefixAndConvertToByteArray(Bytes prefixedTriple) {
    --- End diff --
    
    requireNonNull(tripleBytes)


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153254712
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    +            LOG.debug("Retrieving Metadata from Cache: {}.", nodeId);
    +            return (JoinMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readJoinMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access JoinMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public FilterMetadata readFilterMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.FILTER);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (FilterMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readFilterMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access FilterMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ProjectionMetadata readProjectionMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        checkArgument(type.isPresent() && type.get() == NodeType.PROJECTION);
    +        LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +        try {
    +            return (ProjectionMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readProjectionMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ProjectionMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public AggregationMetadata readAggregationMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.AGGREGATION);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (AggregationMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readAggregationMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access AggregationMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public ConstructQueryMetadata readConstructQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.CONSTRUCT);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (ConstructQueryMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}.", nodeId);
    +                    return dao.readConstructQueryMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access ConstructQueryMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public PeriodicQueryMetadata readPeriodicQueryMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.PERIODIC_QUERY);
    --- End diff --
    
    same....I'm just going to stop and wait to see if this actually gets caught


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153574297
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/IncUpdateDAO.java ---
    @@ -42,7 +43,8 @@
         private static final WholeRowTripleResolver tr = new WholeRowTripleResolver();
     
         public static RyaStatement deserializeTriple(final Bytes row) {
    -        final byte[] rowArray = row.toArray();
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153264201
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/IncUpdateDAO.java ---
    @@ -42,7 +43,8 @@
         private static final WholeRowTripleResolver tr = new WholeRowTripleResolver();
     
         public static RyaStatement deserializeTriple(final Bytes row) {
    -        final byte[] rowArray = row.toArray();
    --- End diff --
    
    Also add documentation to this method.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153630245
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274991
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    --- End diff --
    
    This comment applies for all the methods.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153578221
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    --- End diff --
    
    Addressed.  Provided a link to BindingHashShardingFunction which documents how sharded row keys are generated.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153563566
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/pom.xml ---
    @@ -1,25 +1,16 @@
     <?xml version="1.0" encoding="utf-8"?>
    -<!--
    -
    -    Licensed to the Apache Software Foundation (ASF) under one
    -    or more contributor license agreements.  See the NOTICE file
    -    distributed with this work for additional information
    -    regarding copyright ownership.  The ASF licenses this file
    -    to you under the Apache License, Version 2.0 (the
    -    "License"); you may not use this file except in compliance
    -    with the License.  You may obtain a copy of the License at
    -
    -        http://www.apache.org/licenses/LICENSE-2.0
    -
    -    Unless required by applicable law or agreed to in writing,
    -    software distributed under the License is distributed on an
    -    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    -    KIND, either express or implied.  See the License for the
    -    specific language governing permissions and limitations
    -    under the License.
    -
    --->
    -<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153261968
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    --- End diff --
    
    doc


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153561824
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -104,7 +107,7 @@ public void deletePcj(final FluoClient client, final String pcjId) throws Unsupp
          * @param tx - Transaction of a given Fluo table. (not null)
          * @param pcjId - Id of query. (not null)
          * @return list of Node IDs associated with the query {@code pcjId}.
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153571640
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153560250
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    --- End diff --
    
    It does both.  I'm assuming that you are suggesting that I change "and" to "or"?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153537270
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -55,11 +54,10 @@
         private static final Logger log = LoggerFactory.getLogger(TripleObserver.class);
     
         private static final VisibilityBindingSetSerDe BS_SERDE = new VisibilityBindingSetSerDe();
    -    private static final FluoQueryMetadataDAO QUERY_METADATA_DAO = new FluoQueryMetadataDAO();
    +    private final FluoQueryMetadataCache QUERY_METADATA_DAO = MetadataCacheSupplier.getOrCreateCache();;
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153252294
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -131,10 +134,15 @@ private void deleteMetadata(final Transaction tx, final List<String> nodeIds, fi
             requireNonNull(pcjId);
     
             try (final Transaction typeTx = tx) {
    --- End diff --
    
    This try block is a little unclear if you are unfamiliar with this code already. Could you add some in line docs indicating the first loop removes the metadata the app uses to evaluate statement patterns, then the second remove call updates the list of SP IDs that are used for whatever they are used for? 


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153562826
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -131,10 +134,15 @@ private void deleteMetadata(final Transaction tx, final List<String> nodeIds, fi
             requireNonNull(pcjId);
     
             try (final Transaction typeTx = tx) {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153008756
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -183,18 +177,18 @@ public void updateJoinResults(
         public static enum Side {
             LEFT, RIGHT;
         }
    -    
    -    
    +
    +
         /**
          * Fetches batch to be processed by scanning over the Span specified by the
          * {@link JoinBatchInformation}. The number of results is less than or equal
          * to the batch size specified by the JoinBatchInformation.
    -     * 
    +     *
          * @param tx - Fluo transaction in which batch operation is performed
          * @param siblingSpan - span of sibling to retrieve elements to join with
          * @param bsSet- set that batch results are added to
          * @return Set - containing results of sibling scan.
    -     * @throws Exception 
    +     * @throws Exception
    --- End diff --
    
    doc when this gets thrown


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153606055
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCache.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.collect.Sets;
    +
    +/**
    + * This class caches the StatementPattern Ids so they don't have
    + * to be looked up each time a new Statement needs to be processed
    + * in the TripleObserver.
    + *
    + */
    +public class StatementPatternIdCache {
    +
    +    private final ReentrantLock lock = new ReentrantLock();
    +    private static Optional<String> HASH;
    +    private static Set<String> IDS;
    +
    +    public StatementPatternIdCache() {
    --- End diff --
    
    Okay.  Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153260742
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    --- End diff --
    
    This method looks a little weird. You don't really need to use a StringBuilder since you're not using loops or anything to build your final string. It's good for avoiding building a bunch of little strings that get composed into a larger string late because Strings are immutable and the builder is not. However, the compiler can figure out simple stuff where there is no looping, so you don't need to use one here.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153256824
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    +        if(other == null) { return false;}
    +        if(this == other) { return true;}
    +
    +        if (other instanceof BindingSetRow) {
    +            BindingSetRow row = (BindingSetRow) other;
    +            return new EqualsBuilder().append(this.nodeId, row.nodeId).append(this.bindingSetString, row.bindingSetString)
    --- End diff --
    
    Could you update this to use java.util.Objects.equals comparisons instead? It doesn't require building an object to evaluate the equality and it is built into Java.
    
    return Objects.equals(nodeId, row.nodeId) && Objects.equals(bindingSetString, row.bindingSetString);


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153266212
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/CreateDeletePeriodicPCJ.java ---
    @@ -109,11 +109,11 @@ public void deletePeriodicPCJ() throws Exception {
                             vf.createLiteral(dtf.newXMLGregorianCalendar(time4))),
                     vf.createStatement(vf.createURI("urn:obs_4"), vf.createURI("uri:hasId"), vf.createLiteral("id_4")));
     
    -        runTest(query, statements, 29);
    +        runTest(query, statements, 30);
    --- End diff --
    
    this test changed?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153535410
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    +     * is not included in the Span.
    +     * @param batchSize - size of batch to be deleted
    +     * @param column - column whose entries will be deleted
    +     * @param span - Span indicating the range of data to delete.  Sometimes the Span cannot contain the hash
    +     * (for example, if you are deleting all of the results associated with a nodeId).  In this case, a nodeId
    +     * should be specified along with a Span equal to the prefix of the nodeId.
    +     */
    +    public SpanBatchDeleteInformation(Optional<String> nodeId, int batchSize, Column column, Span span) {
             super(batchSize, Task.Delete, column, span);
    +        Preconditions.checkNotNull(nodeId);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263260
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    --- End diff --
    
    You shouldn't link to a private field within a method's documentation. You could probably just say "Prepends the triple prefix to the provided bytes and returns the new values as a {@link Bytes}.


---

[GitHub] incubator-rya issue #251: Candidate/rya 406

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

    https://github.com/apache/incubator-rya/pull/251
  
    
    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/557/<h2>Build result: FAILURE</span></h2>[...truncated 367.34 KB...][INFO] Apache Rya Web Projects ............................ SKIPPED[INFO] Apache Rya Web Implementation ...................... SKIPPED[INFO] ------------------------------------------------------------------------[INFO] BUILD FAILURE[INFO] ------------------------------------------------------------------------[INFO] Total time: 06:23 min[INFO] Finished at: 2017-11-28T21:50:26+00:00[INFO] Final Memory: 162M/3039M[INFO] ------------------------------------------------------------------------[ERROR] Failed to execute goal org.apache.maven.plugins:maven-checkstyle-plugin:2.17:check (check-style) on project rya.pcj.fluo.app: You have 2 Checkstyle violations. -> [Help 1][ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.[ERROR] Re-run Maven using the -X switch to enable full debug logging.[ERROR] [ER
 ROR] For more information about the errors and possible solutions, please read the following articles:[ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException[ERROR] [ERROR] After correcting the problems, you can resume the build with the command[ERROR]   mvn <goals> -rf :rya.pcj.fluo.appchannel stoppedSetting status of b216522b1f1db4dd221f675f8a5e96d12357040a to FAILURE with url https://builds.apache.org/job/incubator-rya-master-with-optionals-pull-requests/557/ and message: 'FAILURE 'Using context: Jenkins: clean package -Pgeoindexing



---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153561298
  
    --- Diff: extras/indexingExample/src/main/java/RyaClientExample.java ---
    @@ -75,7 +76,7 @@
         private static final Logger log = Logger.getLogger(RyaClientExample.class);
     
         public static void main(final String[] args) throws Exception {
    -        setupLogging();
    +//        setupLogging();
    --- End diff --
    
    Removed.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153529877
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/GetQueryReport.java ---
    @@ -132,14 +135,20 @@ private BigInteger countBindingSets(final SnapshotBase sx, final String nodeId,
             checkNotNull(nodeId);
             checkNotNull(bindingSetColumn);
     
    +        NodeType type = NodeType.fromNodeId(nodeId).get();
    +        Bytes prefixBytes = Bytes.of(type.getNodeTypePrefix());
    +
             // Limit the scan to the binding set column and node id.
    -        final RowScanner rows = sx.scanner().over(Span.prefix(nodeId)).fetch(bindingSetColumn).byRow().build();
    --- End diff --
    
    Sharding changes how binding set results can be accessed in the Fluo table.  There is no concern about breaking existing Fluo instances because the instances can't be updated.  Just redeployed with an updated jar.  


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153534997
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java ---
    @@ -101,14 +99,14 @@ public void processBatch(TransactionBase tx, Bytes row, BatchInformation batch)
             }
     
             // Insert the new join binding sets to the Fluo table.
    -        final JoinMetadata joinMetadata = dao.readJoinMetadata(tx, nodeId);
    +        final JoinMetadata joinMetadata = CACHE.readJoinMetadata(tx, nodeId);
             final VariableOrder joinVarOrder = joinMetadata.getVariableOrder();
             while (newJoinResults.hasNext()) {
                 final VisibilityBindingSet newJoinResult = newJoinResults.next();
                 //create BindingSet value
                 Bytes bsBytes = BS_SERDE.serialize(newJoinResult);
                 //make rowId
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153007773
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/InsertTriples.java ---
    @@ -116,7 +116,7 @@ public void insert(final FluoClient fluo, final Collection<RyaStatement> triples
                 for(final RyaStatement triple : triples) {
                     Optional<byte[]> visibility = Optional.fromNullable(triple.getColumnVisibility());
                     try {
    -                    tx.set(Bytes.of(spoFormat(triple)), FluoQueryColumns.TRIPLES, Bytes.of(visibility.or(new byte[0])));
    +                    tx.set(spoFormat(triple), FluoQueryColumns.TRIPLES, Bytes.of(visibility.or(new byte[0])));
    --- End diff --
    
    the new byte[0] looks to be an empty visibility?  Could you bring this up to a final static class var, this might also be something nice to see in the ColumnVisibility class


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153274914
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    --- End diff --
    
    Null checks.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153631342
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/CreateDeletePeriodicPCJ.java ---
    @@ -109,11 +109,11 @@ public void deletePeriodicPCJ() throws Exception {
                             vf.createLiteral(dtf.newXMLGregorianCalendar(time4))),
                     vf.createStatement(vf.createURI("urn:obs_4"), vf.createURI("uri:hasId"), vf.createLiteral("id_4")));
     
    -        runTest(query, statements, 29);
    +        runTest(query, statements, 30);
    --- End diff --
    
    Yes.  Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153570806
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -21,15 +21,16 @@
     import static com.google.common.base.Preconditions.checkNotNull;
     import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
     
    +import org.apache.commons.lang.builder.EqualsBuilder;
     import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
     
     import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
     import edu.umd.cs.findbugs.annotations.NonNull;
     import net.jcip.annotations.Immutable;
     
     /**
    - * The values of an Accumulo Row ID for a row that stores a Binding set for
    - * a specific Node ID of a query.
    + * The values of an Accumulo Row ID for a row that stores a Binding set for a specific Node ID of a query.
    --- End diff --
    
    I don't think this is the appropriate place to document this.  I added documentation to the class BindingHashShardingFunction about how the sharded row id is created and linked this class to the makeFromShardRow(...) method in BindingSetRow.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153575097
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153575392
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java ---
    @@ -231,30 +225,38 @@ public void updateJoinResults(
          * @param siblingId - Id of the sibling node whose BindingSets will be retrieved and joined with the update
          * @return Span to retrieve sibling node's BindingSets to form join results
          */
    -    private Span getSpan(TransactionBase tx, final String childId, final BindingSet childBindingSet, final String siblingId) {
    +    private Span getSpan(TransactionBase tx, final String childId, final VisibilityBindingSet childBindingSet, final String siblingId) {
             // Get the common variable orders. These are used to build the prefix.
             final VariableOrder childVarOrder = getVarOrder(tx, childId);
             final VariableOrder siblingVarOrder = getVarOrder(tx, siblingId);
             final List<String> commonVars = getCommonVars(childVarOrder, siblingVarOrder);
     
    -        // Get the Binding strings
    -        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    -        final String[] childBindingArray = childBindingSetString.split("\u0001");
    -        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    -
    -        // Create the prefix that will be used to scan for binding sets of the sibling node.
    -        // This prefix includes the sibling Node ID and the common variable values from
    -        // childBindingSet.
    -        String siblingScanPrefix = "";
    -        for(int i = 0; i < commonVars.size(); i++) {
    -            if(siblingScanPrefix.length() == 0) {
    -                siblingScanPrefix = childBindingStrings[i];
    -            } else {
    -                siblingScanPrefix += DELIM + childBindingStrings[i];
    -            }
    +//        // Get the Binding strings
    +//        final String childBindingSetString = VIS_BS_CONVERTER.convert(childBindingSet, childVarOrder);
    +//        final String[] childBindingArray = childBindingSetString.split("\u0001");
    +//        final String[] childBindingStrings = FluoStringConverter.toBindingStrings(childBindingArray[0]);
    +
    +        Bytes scanPrefix = null;
    +        if(!commonVars.isEmpty()) {
    +            scanPrefix = getRowKey(siblingId, new VariableOrder(commonVars), childBindingSet);
    +        } else {
    +            scanPrefix = getRowKey(siblingId, siblingVarOrder, childBindingSet);
             }
    -        siblingScanPrefix = siblingId + NODEID_BS_DELIM + siblingScanPrefix;
    -        return Span.prefix(siblingScanPrefix);
    +//
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153262981
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    +     * @param ids - ids to add to the StatementPattern nodeId Set
    +     */
    +    public static void addStatementPatternIds(TransactionBase tx, Set<String> ids) {
    +        Optional<Bytes> val = Optional.fromNullable(tx.get(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS));
    +        StringBuilder builder = new StringBuilder();
    +        if (val.isPresent()) {
    +            builder.append(val.get().toString());
    +            builder.append(VAR_DELIM);
    +        }
    +        String idString = builder.append(Joiner.on(VAR_DELIM).join(ids)).toString();
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS, Bytes.of(idString));
    +        tx.set(Bytes.of(STATEMENT_PATTERN_ID), STATEMENT_PATTERN_IDS_HASH, Bytes.of(Hashing.sha256().hashString(idString).toString()));
    +    }
    +
    +    /**
    +     * Remove specified Set of ids from the Fluo table and updates the entry with Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also updates the hash of the updated nodeId Set and writes that
    +     * to the Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    --- End diff --
    
    doc for tx


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153252695
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    +                @Override
    +                public CommonNodeMetadata call() throws Exception {
    +                    LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
    +                    return dao.readStatementPatternMetadata(tx, nodeId);
    +                }
    +            });
    +        } catch (Exception e) {
    +            throw new RuntimeException("Unable to access StatementPatternMetadata for nodeId: " + nodeId, e);
    +        }
    +    }
    +
    +    @Override
    +    public JoinMetadata readJoinMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.JOIN);
    --- End diff --
    
    same


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153626670
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdManager.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.STATEMENT_PATTERN_ID;
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.VAR_DELIM;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS;
    +import static org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns.STATEMENT_PATTERN_IDS_HASH;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +import org.apache.fluo.api.client.TransactionBase;
    +import org.apache.fluo.api.data.Bytes;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Optional;
    +import com.google.common.collect.Sets;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * Utility class for updating and removing StatementPattern nodeIds in the Fluo table. All StatementPattern nodeIds are
    + * stored in a single set under a single entry in the Fluo table. This is to eliminate the need for a scan to find all
    + * StatementPattern nodeIds every time a new Triple enters the Fluo table. Instead, the nodeIds are cached locally, and
    + * only updated when the local hash of the nodeId set is dirty (not equal to the hash in the Fluo table).
    + */
    +public class StatementPatternIdManager {
    +
    +    /**
    +     * Add specified Set of ids to the Fluo table with Column {@link FluoQueryColumns#STATEMENT_PATTERN_IDS}. Also
    +     * updates the hash of the updated nodeId Set and writes that to the Column
    +     * {@link FluoQueryColumns#STATEMENT_PATTERN_IDS_HASH}
    +     *
    +     * @param tx
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153589770
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/MetadataCacheSupplier.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class MetadataCacheSupplier {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(MetadataCacheSupplier.class);
    +    private static FluoQueryMetadataCache CACHE;
    +    private static boolean initialized = false;
    +    private static final int DEFAULT_CAPACITY = 10000;
    +    private static final int DEFAULT_CONCURRENCY = 8;
    +    private static final ReentrantLock lock = new ReentrantLock();
    +
    +    /**
    +     * Returns an existing cache with the specified instance name, or creates a cache. The created cache will have the
    +     * indicated capacity and concurrencyLevel if one is provided.
    +     *
    +     * @param capacity - capacity used to create a new cache
    +     * @param concurrencyLevel - concurrencyLevel used to create a new cache
    +     */
    +    public static FluoQueryMetadataCache getOrCreateCache(int capacity, int concurrencyLevel) {
    +        lock.lock();
    +        try {
    +            if (!initialized) {
    +                LOG.debug("Cache has not been initialized.  Initializing cache with capacity: {} and concurrencylevel: {}", capacity,
    +                        concurrencyLevel);
    +                CACHE = new FluoQueryMetadataCache(new FluoQueryMetadataDAO(), capacity, concurrencyLevel);
    +                initialized = true;
    +            } else {
    +                LOG.warn(
    +                        "A cache has already been initialized, so a cache with capacity: {} and concurrency level: {} will not be created.  Returning existing cache with capacity: {} and concurrencylevel: {}",
    +                        capacity, concurrencyLevel, CACHE.getCapacity(), CACHE.getConcurrencyLevel());
    +            }
    +            return CACHE;
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    /**
    +     * Returns cache with the name {@link FluoQueryMetadataCache#FLUO_CACHE_INSTANCE} if it exists, otherwise creates it
    --- End diff --
    
    Updated.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153577373
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153629576
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r154195144
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AbstractNodeUpdater.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public abstract class AbstractNodeUpdater {
    --- End diff --
    
    I disagree that it effects the number of of files that would have to be updated. You still have to change either the code within that function to always use a different strategy (which would be the same as the utility function), or you would have to pass the function into the parent via a constructor or method, which still means you have to touch all of the child files again.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153007455
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/GetQueryReport.java ---
    @@ -132,14 +135,20 @@ private BigInteger countBindingSets(final SnapshotBase sx, final String nodeId,
             checkNotNull(nodeId);
             checkNotNull(bindingSetColumn);
     
    +        NodeType type = NodeType.fromNodeId(nodeId).get();
    +        Bytes prefixBytes = Bytes.of(type.getNodeTypePrefix());
    +
             // Limit the scan to the binding set column and node id.
    -        final RowScanner rows = sx.scanner().over(Span.prefix(nodeId)).fetch(bindingSetColumn).byRow().build();
    --- End diff --
    
    is this changing the actual query?  using nodeID vs prefixBytes.  Will this break an existing fluo instance?


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153009029
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/QueryResultUpdater.java ---
    @@ -70,7 +69,7 @@ public void updateQueryResults(
             final VariableOrder queryVarOrder = queryMetadata.getVariableOrder();
     
             // Create the Row Key for the result. If the child node groups results, then the key must only contain the Group By variables.
    -        final Bytes resultRow = RowKeyUtil.makeRowKey(queryMetadata.getNodeId(), queryVarOrder, childBindingSet);
    +        final Bytes resultRow = getRowKey(queryMetadata.getNodeId(), queryVarOrder, childBindingSet);
    --- End diff --
    
    create vs get


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153561578
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/DeleteFluoPcj.java ---
    @@ -79,7 +82,7 @@ public DeleteFluoPcj(final int batchSize) {
          *            Index. (not null)
          * @param pcjId - The PCJ ID for the query that will removed from the Fluo
          *            application. (not null)
    -     * @throws UnsupportedQueryException 
    +     * @throws UnsupportedQueryException
    --- End diff --
    
    Done.  Though, the Exception is well documented in itself and provides more explanation than a method header would.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263555
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    --- End diff --
    
    Nullness contract.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153254432
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AbstractNodeUpdater.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.rya.indexing.pcj.fluo.app.util.BindingHashShardingFunction;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +
    +public abstract class AbstractNodeUpdater {
    --- End diff --
    
    This class needs documentation.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153275576
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    +            LOG.debug("Retrieving Metadata from Cache: {}", nodeId);
    +            return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, new Callable<CommonNodeMetadata>() {
    --- End diff --
    
    Could shorten these with a lambda through out this class.
    
    ```java
    return (StatementPatternMetadata) commonNodeMetadataCache.get(nodeId, () -> {
        LOG.debug("Seeking Metadata from Fluo Table: {}", nodeId);
        return dao.readStatementPatternMetadata(tx, nodeId);
    });
    ```


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153584456
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153255918
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    --- End diff --
    
    You need to implement the hashCode() method of this class since you've written an equality method.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153573688
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    +        return make(BindingHashShardingFunction.removeHash(prefixBytes, row));
    +    }
    +
    +    @Override
    +    public String toString() {
    +        StringBuilder builder = new StringBuilder();
    +        builder
    +        .append("NodeId: " + nodeId).append("\n")
    +        .append("BindingSet String: " + bindingSetString);
    +        return builder.toString();
    +    }
    +
    +    @Override
    +    public boolean equals(Object other) {
    +        if(other == null) { return false;}
    +        if(this == other) { return true;}
    +
    +        if (other instanceof BindingSetRow) {
    +            BindingSetRow row = (BindingSetRow) other;
    +            return new EqualsBuilder().append(this.nodeId, row.nodeId).append(this.bindingSetString, row.bindingSetString)
    --- End diff --
    
    Okay, but only because you were nice enough to code it up for me :)


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153265659
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/AbstractBatchBindingSetUpdater.java ---
    @@ -23,13 +23,17 @@
     import org.apache.fluo.api.data.RowColumn;
     import org.apache.fluo.api.data.Span;
     import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;
    +import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryMetadataCache;
    +import org.apache.rya.indexing.pcj.fluo.app.query.MetadataCacheSupplier;
     
     /**
      * This class provides common functionality for implementations of {@link BatchBindingSetUpdater}.
      *
      */
     public abstract class AbstractBatchBindingSetUpdater implements BatchBindingSetUpdater {
     
    +    protected static final FluoQueryMetadataCache CACHE = MetadataCacheSupplier.getOrCreateCache();
    --- End diff --
    
    Add some documentation to this object so that people who extend this class know when it is appropriate to use it.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153263671
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/TriplePrefixUtils.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaStatement;
    +
    +/**
    + * This class is a utility class for adding and removing the Triple prefix to
    + * Statements ingested into the Rya Fluo application.  The Triple prefix is added
    + * to supported range based compactions for removing transient data from the Fluo
    + * application.  This prefix supports the Transient data recipe described on the
    + * the Fluo website, and reduces the computational load on the system by cleaning up
    + * old deleted Triples and notifications using a targeted range compaction.
    + *
    + */
    +public class TriplePrefixUtils {
    +
    +    private static final Bytes TRIPLE_PREFIX_BYTES = Bytes.of("T" + NODEID_BS_DELIM);
    +
    +    /**
    +     * Adds the prefix {@link TriplePrefixUtils#TRIPLE_PREFIX_BYTES} and converts to {@link Bytes}.
    +     * @param tripleBytes - serialized {@link RyaStatement}
    +     * @return - serialized RyaStatement with prepended triple prefix, converted to Bytes
    +     */
    +    public static Bytes addTriplePrefixAndConvertToBytes(byte[] tripleBytes) {
    --- End diff --
    
    requireNonNull(tripleBytes)


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153561208
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/CreateDeletePeriodicPCJ.java ---
    @@ -109,11 +109,11 @@ public void deletePeriodicPCJ() throws Exception {
                             vf.createLiteral(dtf.newXMLGregorianCalendar(time4))),
                     vf.createStatement(vf.createURI("urn:obs_4"), vf.createURI("uri:hasId"), vf.createLiteral("id_4")));
     
    -        runTest(query, statements, 29);
    +        runTest(query, statements, 30);
    --- End diff --
    
    Yep.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153030940
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/SpanBatchDeleteInformation.java ---
    @@ -20,39 +22,79 @@
     import org.apache.fluo.api.data.Column;
     import org.apache.fluo.api.data.Span;
     
    +import com.google.common.base.Preconditions;
    +
     /**
      * This class represents a batch order to delete all entries in the Fluo table indicated
      * by the given Span and Column.  These batch orders are processed by the {@link BatchObserver},
      * which uses this batch information along with the nodeId passed into the Observer to perform
    - * batch deletes.  
    + * batch deletes.
      *
      */
     public class SpanBatchDeleteInformation extends AbstractSpanBatchInformation {
     
         private static final BatchBindingSetUpdater updater = new SpanBatchBindingSetUpdater();
    -    
    -    public SpanBatchDeleteInformation(int batchSize, Column column, Span span) {
    +    private Optional<String> nodeId;
    +
    +    /**
    +     * Create a new SpanBatchInformation object.
    +     * @param nodeId - Optional nodeId that is used to filter returned results.  Useful if the hash
    +     * is not included in the Span.
    +     * @param batchSize - size of batch to be deleted
    +     * @param column - column whose entries will be deleted
    +     * @param span - Span indicating the range of data to delete.  Sometimes the Span cannot contain the hash
    +     * (for example, if you are deleting all of the results associated with a nodeId).  In this case, a nodeId
    +     * should be specified along with a Span equal to the prefix of the nodeId.
    +     */
    +    public SpanBatchDeleteInformation(Optional<String> nodeId, int batchSize, Column column, Span span) {
             super(batchSize, Task.Delete, column, span);
    +        Preconditions.checkNotNull(nodeId);
    --- End diff --
    
    static import


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153273836
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    --- End diff --
    
    Style nit: I usually separate fields that hold the parameters that were passed in through the constructor from the rest of the instance fields because it's a little easier to see what wasn't passed in.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153586913
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataCache.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +
    +import java.util.concurrent.Callable;
    +
    +import org.apache.fluo.api.client.SnapshotBase;
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Column;
    +import org.apache.rya.indexing.pcj.fluo.app.NodeType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +
    +/**
    + * Wrapper for {@link FluoQueryMetadataDAO} that caches any metadata that has been retrieved from Fluo. This class first
    + * checks the cache to see if the metadata is present before delegating to the underlying DAO method to retrieve the
    + * data.
    + *
    + */
    +public class FluoQueryMetadataCache extends FluoQueryMetadataDAO {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(FluoQueryMetadataCache.class);
    +    private final FluoQueryMetadataDAO dao;
    +    private final Cache<String, CommonNodeMetadata> commonNodeMetadataCache;
    +    private final Cache<String, Bytes> metadataCache;
    +    private int capacity;
    +    private int concurrencyLevel;
    +
    +    /**
    +     * Creates a FluoQueryMetadataCache with the specified capacity. Old, unused results are evicted as necessary.
    +     *
    +     * @param capacity - max size of the cache
    +     */
    +    public FluoQueryMetadataCache(FluoQueryMetadataDAO dao, int capacity, int concurrencyLevel) {
    +        this.dao = dao;
    +        commonNodeMetadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        metadataCache = CacheBuilder.newBuilder().concurrencyLevel(concurrencyLevel).maximumSize(capacity).build();
    +        this.capacity = capacity;
    +        this.concurrencyLevel = concurrencyLevel;
    +    }
    +
    +    /**
    +     * @return - capacity of this cache in terms of max number of entries
    +     */
    +    public int getCapacity() {
    +        return capacity;
    +    }
    +
    +    /**
    +     * @return - concurrencyLevel of this cache,in terms of number of partitions that distinct threads can operate on
    +     *         without waiting for other threads
    +     */
    +    public int getConcurrencyLevel() {
    +        return concurrencyLevel;
    +    }
    +
    +
    +    @Override
    +    public StatementPatternMetadata readStatementPatternMetadata(SnapshotBase tx, String nodeId) {
    +        Optional<NodeType> type = NodeType.fromNodeId(nodeId);
    +
    +        try {
    +            checkArgument(type.isPresent() && type.get() == NodeType.STATEMENT_PATTERN);
    --- End diff --
    
    Done.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153558832
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/StatementPatternIdCacheSupplier.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.query;
    +
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StatementPatternIdCacheSupplier {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(StatementPatternIdCacheSupplier.class);
    +    private static boolean initialized = false;
    +    private static StatementPatternIdCache CACHE;
    +    private static final ReentrantLock lock = new ReentrantLock();
    +
    +    /**
    +     * Returns an existing cache if one has been created, otherwise creates a new cache.
    +     *
    +     * @return - existing StatementPatternIdCache or new cache if one didn't already exist
    +     */
    +    public static StatementPatternIdCache getOrCreateCache() {
    +        lock.lock();
    +        try {
    +            if (!initialized) {
    +                LOG.debug("Cache has not been initialized.  Initializing StatementPatternIdCache");
    +                CACHE = new StatementPatternIdCache();
    +                initialized = true;
    +            } else {
    +                LOG.debug("A StatementPatternIdCache has already been initialized.");
    +            }
    +            return CACHE;
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    /**
    +     * Flushes stored cache and flags Supplier as uninitialized.
    --- End diff --
    
    I think clear is more consistent with Java method names.  For example, clear() is the method used to delete the contents of hash tables, sets, etc.


---

[GitHub] incubator-rya issue #251: Candidate/rya 406

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

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



---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153264159
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/IncUpdateDAO.java ---
    @@ -42,7 +43,8 @@
         private static final WholeRowTripleResolver tr = new WholeRowTripleResolver();
     
         public static RyaStatement deserializeTriple(final Bytes row) {
    -        final byte[] rowArray = row.toArray();
    --- End diff --
    
    requireNonNull(row);


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153271360
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/TripleObserver.java ---
    @@ -70,53 +68,50 @@ public void process(final TransactionBase tx, final Bytes brow, final Column col
             // Get string representation of triple.
             final RyaStatement ryaStatement = IncUpdateDAO.deserializeTriple(brow);
             log.trace("Transaction ID: {}\nRya Statement: {}\n", tx.getStartTimestamp(), ryaStatement);
    +        log.trace("Beginging to process triple.");
     
             final String triple = IncUpdateDAO.getTripleString(ryaStatement);
     
    -        // Iterate over each of the Statement Patterns that are being matched against.
    -        final RowScanner spScanner = tx.scanner()
    -                .over(Span.prefix(SP_PREFIX))
    -
    -                // Only fetch rows that have the pattern in them. There will only be a single row with a pattern per SP.
    -                .fetch(FluoQueryColumns.STATEMENT_PATTERN_PATTERN)
    -                .byRow()
    -                .build();
    +        Set<String> spIDs = SP_ID_CACHE.getStatementPatternIds(tx);
     
             //see if triple matches conditions of any of the SP
    -        for (final ColumnScanner colScanner : spScanner) {
    -            // Get the Statement Pattern's node id.
    -            final String spID = colScanner.getsRow();
    -
    +        for (String spID: spIDs) {
                 // Fetch its metadata.
                 final StatementPatternMetadata spMetadata = QUERY_METADATA_DAO.readStatementPatternMetadata(tx, spID);
     
    +            log.trace("Retrieved metadata: {}", spMetadata);
    +
                 // Attempt to match the triple against the pattern.
                 final String pattern = spMetadata.getStatementPattern();
                 final VariableOrder varOrder = spMetadata.getVariableOrder();
                 final String bindingSetString = getBindingSet(triple, pattern, varOrder);
     
    +            log.trace("Created binding set match string: {}", bindingSetString);
    +
                 // Statement matches to a binding set.
                 if(bindingSetString.length() != 0) {
                     // Fetch the triple's visibility label.
                     final String visibility = tx.gets(brow.toString(), FluoQueryColumns.TRIPLES, "");
     
                     // Create the Row ID for the emitted binding set. It does not contain visibilities.
    -                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    -                final Bytes rowBytes = Bytes.of( row.getBytes(Charsets.UTF_8) );
    +//                final String row = spID + NODEID_BS_DELIM + bindingSetString;
    --- End diff --
    
    Remove commented out code.


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153264283
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/util/BindingHashShardingFunction.java ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.indexing.pcj.fluo.app.util;
    +
    +import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.NODEID_BS_DELIM;
    +import static org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter.TYPE_DELIM;
    +
    +import org.apache.fluo.api.data.Bytes;
    +import org.apache.fluo.api.data.Bytes.BytesBuilder;
    +import org.apache.rya.api.domain.RyaType;
    +import org.apache.rya.api.resolver.RdfToRyaConversions;
    +import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetStringConverter;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
    +import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSet;
    +import org.openrdf.model.Value;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Strings;
    +import com.google.common.hash.Hashing;
    +
    +/**
    + * This class adds and removes a hash to and from the rowId for sharding purposes.
    + *
    + */
    +public class BindingHashShardingFunction {
    +
    +    private static final BindingSetStringConverter BS_CONVERTER = new BindingSetStringConverter();
    +    private static final int HASH_LEN = 4;
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param varOrder - VarOrder used to order BindingSet values
    +     * @param bs - BindingSet with partially formed query values
    +     * @return - serialized Bytes rowId for storing BindingSet results in Fluo
    +     */
    +    public static Bytes addShard(String nodeId, VariableOrder varOrder, VisibilityBindingSet bs) {
    +        String[] rowPrefixAndId = nodeId.split("_");
    +        Preconditions.checkArgument(rowPrefixAndId.length == 2);
    +        String prefix = rowPrefixAndId[0];
    +        String id = rowPrefixAndId[1];
    +
    +        String firstBindingString = "";
    +        Bytes rowSuffix = Bytes.of(id);
    +        if (varOrder.getVariableOrders().size() > 0) {
    +            VariableOrder first = new VariableOrder(varOrder.getVariableOrders().get(0));
    +            firstBindingString = BS_CONVERTER.convert(bs, first);
    +            rowSuffix = RowKeyUtil.makeRowKey(id, varOrder, bs);
    +        }
    +
    +        BytesBuilder builder = Bytes.builder();
    +        builder.append(Bytes.of(prefix + ":"));
    +        builder.append(genHash(Bytes.of(id + NODEID_BS_DELIM + firstBindingString)));
    +        builder.append(":");
    +        builder.append(rowSuffix);
    +        return builder.toBytes();
    +    }
    +
    +    /**
    +     * Generates a sharded rowId.
    +     *
    +     * @param nodeId - Node Id with type and UUID
    +     * @param firstBsVal - String representation of the first BsValue
    +     * @return - serialized Bytes prefix for scanning rows
    +     */
    +    public static Bytes getShardedScanPrefix(String nodeId, Value firstBsVal) {
    +        Preconditions.checkNotNull(firstBsVal);
    +
    +        final RyaType ryaValue = RdfToRyaConversions.convertValue(firstBsVal);
    +        final String bindingString = ryaValue.getData() + TYPE_DELIM + ryaValue.getDataType();
    +
    +        return getShardedScanPrefix(nodeId, bindingString);
    +    }
    +
    +    /**
    +     * Generates a sharded rowId from the indicated nodeId and bindingString.
    +     *
    +     * @param nodeId - NodeId with tyep and UUID
    +     * @param bindingString - String representation of BindingSet values, as formed by {@link BindingSetStringConverter}
    +     *            .
    --- End diff --
    
    odd '.' here


---

[GitHub] incubator-rya pull request #251: Candidate/rya 406

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/251#discussion_r153256113
  
    --- Diff: extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/BindingSetRow.java ---
    @@ -77,4 +78,31 @@ public static BindingSetRow make(final Bytes row) {
             final String bindingSetString = rowArray.length == 2 ? rowArray[1] : "";
             return new BindingSetRow(nodeId, bindingSetString);
         }
    +
    +    public static BindingSetRow makeFromShardedRow(Bytes prefixBytes, Bytes row) {
    --- End diff --
    
    Documentation.


---